diff --git a/.rat-excludes b/.rat-excludes
index b14ad53720f32d837eac64b8240930a2c00f4712..ae9745673c87d7ad36337f2feedc8d0d0bd2e67b 100644
--- a/.rat-excludes
+++ b/.rat-excludes
@@ -48,6 +48,7 @@ sbt-launch-lib.bash
 plugins.sbt
 work
 .*\.q
+.*\.qv
 golden
 test.out/*
 .*iml
diff --git a/dev/run-tests b/dev/run-tests
index 7d06c86eb4b4156a137b79ba87184c337b38a8fd..f55497ae2bfbd7c470ef6112dac03f4187d4ba90 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
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 705937e3016e2cdaee232c926ad7e05c3399965b..ea044738540071a782d10a4f35ed95c592ff701a 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") }
   )
 
 }
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 7c480de107e7f93d85c621415d66b72724eb0c8a..2b69c02b28285b4801a70c07a701b9a3f65892e8 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))
     }
   }
 
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 dafd745ec96c6b70f3ee28a620dc355a67ff5cbf..19421e5667138b906a6378657f0f462365f7f580 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(",")})"
+}
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 590dbf3cb893d38e90be532ae2678f654ea8569a..c4f4ef01d78dfb64ab24b8c80b91c29ff5a34718 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
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 463888551a359d92bbab841f0c6c0e4751ac48d3..15cd62d3bf869c3f73bbda66041dddbf08523af6 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",
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 34ed57b0016374b08ab467252d979a09cf12b1d2..fad4091d48a89cad752664c0af10dfd344ab6a5a 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.
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 deaa1a2a154f2e79c3bf3250dcbd1022c7a4b4da..fad7373a2fa398a2e5e22b416b459026e610bfb9 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._
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 904bb48691e35b79d3d0ed470ae385551a431ffb..04c48c385966e78c2505aeb811c13826b0f0ac89 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 */
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 ffcb6b505b9c681e966c79815b66fc010b79e900..54c619722ee126d357d3501aa313e8f469d8b4a4 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
       }
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 e45eb57b3debf759bc137c6fcb992520e717eef3..9ff7ab5a124c1ca48bbbe06bb7787b1b883a1197 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.
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 c6ff4ea6de594abfdc033451d34b46057e16129c..bb79ad5538046191be6576ce03dcaafa644d6925 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")
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 a201d2349a2efcf6ace98ac92ef18314cca50996..1817c7832490e0944d169b4aa1a3b1bcd46825b5 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.
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 fbd375639692ff892ff7da988057828478914d30..5d98834c6fb33b97b740d5b45bd415926651322c 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")
   }
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 4e6ff16135833da1a4df12a12a6fe59ad4f870ba..7931d6a7128fe32e25cd37f80bb1bd1378c5d7ab 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,22 +48,27 @@
 
 <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>
@@ -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>
 
@@ -143,18 +141,25 @@
   <description>Post Execute Hook for Tests</description>
 </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>
@@ -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>
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 0000000000000000000000000000000000000000..96eca8f61de33d5b52fb6b0f5c603c7356ba11b1
--- /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   |
++-------+-----+
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 0000000000000000000000000000000000000000..358cf400ec81575a658653faa1601dce5f6b4f32
--- /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"}
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 0000000000000000000000000000000000000000..c6a05a1dad893b3f7354d0025ec592e5d9b6f4b5
--- /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"}
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 0000000000000000000000000000000000000000..95c68cabd15e0ce34d472d552f4bb60193614b6d
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/alltypesorc differ
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 0000000000000000000000000000000000000000..570078d90e65bb2f710f95c6df63ef8017e435c4
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/char_varchar_udf.txt
@@ -0,0 +1 @@
+47.3224	2923.29346
diff --git a/sql/hive/src/test/resources/data/files/datatypes.txt b/sql/hive/src/test/resources/data/files/datatypes.txt
index 10daa1b11c26b2d6f12a4cccbac11adf2813fcb2..0228a27e927bbdfeb4bbba34df286a811ea276fe 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'
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 0000000000000000000000000000000000000000..28800f5278b106ad63713aaedf89597c78e77c31
--- /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
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 0000000000000000000000000000000000000000..292bee6ee0370f78b72c8ccaf45a909f83e5d52f
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/dept.txt
@@ -0,0 +1,4 @@
+31|sales
+33|engineering
+34|clerical
+35|marketing
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 0000000000000000000000000000000000000000..a0e76b90e57dc8f486194d30a1a61b14b9426cd7
--- /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|
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 0000000000000000000000000000000000000000..81fbf63a549800a660760a862f5c85a49a775262
--- /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
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 0000000000000000000000000000000000000000..40a75acfa001642570ef61c24390f1e7eb46226f
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/exported_table/data/data
@@ -0,0 +1,2 @@
+johndee,1
+burks,2
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 0000000000000000000000000000000000000000..6a3906944cbd181c05b0a02ee8d26f9d958eb7d1
--- /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
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 0000000000000000000000000000000000000000..c08b97b46e3f1334087b6563dc766060361e43b2
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/futurama_episodes.avro differ
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 0000000000000000000000000000000000000000..c242b42b6070bdcaef8e873895096e813acc35a6
--- /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
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 0000000000000000000000000000000000000000..d5db38d168be7982a0221efd733069e4581ba639
--- /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
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 0000000000000000000000000000000000000000..f7a763d8b96388b523ede349549186e35d1bf15f
--- /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
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 0000000000000000000000000000000000000000..c242b42b6070bdcaef8e873895096e813acc35a6
--- /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
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 0000000000000000000000000000000000000000..d5db38d168be7982a0221efd733069e4581ba639
--- /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
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 0000000000000000000000000000000000000000..f7a763d8b96388b523ede349549186e35d1bf15f
--- /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
diff --git a/sql/hive/src/test/resources/golden/alter3-1-75be487df30e301e156a22eee075633d b/sql/hive/src/test/resources/data/files/header_footer_table_3/empty1.txt
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter3-1-75be487df30e301e156a22eee075633d
rename to sql/hive/src/test/resources/data/files/header_footer_table_3/empty1.txt
diff --git a/sql/hive/src/test/resources/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 b/sql/hive/src/test/resources/data/files/header_footer_table_3/empty2.txt
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5
rename to sql/hive/src/test/resources/data/files/header_footer_table_3/empty2.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 0000000000000000000000000000000000000000..caea9919d1d7edbcc55e4f4224d876fcc7a12929
--- /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
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 0000000000000000000000000000000000000000..469d8a543a4d2a94535996ceee2a24891699b1cc
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/keystore.jks differ
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 0000000000000000000000000000000000000000..b72475f21bff060a68d800057d824be3cf38c6ed
--- /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
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 0000000000000000000000000000000000000000..69910b76495710e50fb67f4ed951673b1ec1ff01
--- /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
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 0000000000000000000000000000000000000000..41586d61eda03d61ed3aeb51e0ea3986ecd0fd90
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/non_ascii_tbl.txt
@@ -0,0 +1 @@
+1|Garçu Kôkaku kidôtai
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 884598981a13cfae7a7bfbbccaedf9d4c5b4083f..ab93c1400769a3a341a4f8fe91f6126d2a93a1fc 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
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 0000000000000000000000000000000000000000..cd145d343104983b4b09603c251ee749e5f82cc7
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/orc_split_elim.orc differ
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 0000000000000000000000000000000000000000..ccd48ee37a61bcd7ffc10df5f240cf1c3c84bb1a
--- /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
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 0000000000000000000000000000000000000000..8f322f3bf385ccd12ce7111e475df6574641dc33
--- /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
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 0000000000000000000000000000000000000000..0be390b38cea197801e3704436c2924817c14216
--- /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
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 c902284c111553cfe38fe6c8f1725baf8a234767..f63d2410885a39d7bf5c72a3dc0a8000bf920b19 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
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 0000000000000000000000000000000000000000..9d2fa32342d53cd6db84caa366575d3cf917c756
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/person+age.txt
@@ -0,0 +1,3 @@
+Sean29
+Tim47
+Pooh21
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 0000000000000000000000000000000000000000..d04778becc38fc9042ff3eee1a3b2a3d8e2d812f
--- /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
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 0000000000000000000000000000000000000000..7b749791ef41926a0791397c55082555007c9f7c
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/sample.json
@@ -0,0 +1 @@
+{"a" : "2" ,"b" : "blah"}
diff --git a/sql/hive/src/test/resources/data/files/symlink1.txt b/sql/hive/src/test/resources/data/files/symlink1.txt
index dc1a7c5682a26eea8d8e9d06f38e361723aa2194..91d734607ea35f11eaffd7ecc63631c74cc5f281 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
diff --git a/sql/hive/src/test/resources/data/files/symlink2.txt b/sql/hive/src/test/resources/data/files/symlink2.txt
index 8436a30adf3662bbfc6b670b1d5c58c9ae935464..487b05e941d4630b48b98b0d4843333489dbb3a4 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
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 0000000000000000000000000000000000000000..9c5d703fba6c8c23bf6ee8f431d6d0fc35bf9ee1
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/truststore.jks differ
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 0000000000000000000000000000000000000000..40e3683dc3d363d77e2ca78b465a7b82b1cbcc98
--- /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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 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
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 d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 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
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 d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 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
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 d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 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
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 d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 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
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 d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 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
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 d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 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
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 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 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
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 d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 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
diff --git a/sql/hive/src/test/resources/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d b/sql/hive/src/test/resources/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d
index 415b19fc3623413817e5ad457fcd8f530c06047a..cd5ac039d67e0bdadb17976e4ac39f0ffe6bb6e4 100644
--- a/sql/hive/src/test/resources/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d	
+++ b/sql/hive/src/test/resources/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d	
@@ -1 +1 @@
-2.0
\ No newline at end of file
+2.0
diff --git a/sql/hive/src/test/resources/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 b/sql/hive/src/test/resources/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1
index d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 100644
--- a/sql/hive/src/test/resources/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1	
+++ b/sql/hive/src/test/resources/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1	
@@ -1 +1 @@
-2
\ No newline at end of file
+2
diff --git a/sql/hive/src/test/resources/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 b/sql/hive/src/test/resources/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80
index d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 100644
--- a/sql/hive/src/test/resources/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80	
+++ b/sql/hive/src/test/resources/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80	
@@ -1 +1 @@
-2
\ No newline at end of file
+2
diff --git a/sql/hive/src/test/resources/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 b/sql/hive/src/test/resources/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5
index d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 100644
--- a/sql/hive/src/test/resources/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5	
+++ b/sql/hive/src/test/resources/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5	
@@ -1 +1 @@
-2
\ No newline at end of file
+2
diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0 b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0
deleted file mode 100644
index 7643569a2c2343d49f7ed0aa3b8ce5c88f05a4b2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0	
+++ /dev/null
@@ -1,309 +0,0 @@
-0.0	0	0	0	3
-2.0	0	2	0	1
-4.0	0	4	0	1
-5.0	0	15	0	3
-8.0	0	8	0	1
-9.0	0	9	0	1
-10.0	0	10	0	1
-11.0	0	11	0	1
-12.0	0	24	0	2
-15.0	0	30	0	2
-17.0	0	17	0	1
-18.0	0	36	0	2
-19.0	0	19	0	1
-20.0	0	20	0	1
-24.0	0	48	0	2
-26.0	0	52	0	2
-27.0	0	27	0	1
-28.0	0	28	0	1
-30.0	0	30	0	1
-33.0	0	33	0	1
-34.0	0	34	0	1
-35.0	0	105	0	3
-37.0	0	74	0	2
-41.0	0	41	0	1
-42.0	0	84	0	2
-43.0	0	43	0	1
-44.0	0	44	0	1
-47.0	0	47	0	1
-51.0	0	102	0	2
-53.0	0	53	0	1
-54.0	0	54	0	1
-57.0	0	57	0	1
-58.0	0	116	0	2
-64.0	0	64	0	1
-65.0	0	65	0	1
-66.0	0	66	0	1
-67.0	0	134	0	2
-69.0	0	69	0	1
-70.0	0	210	0	3
-72.0	0	144	0	2
-74.0	0	74	0	1
-76.0	0	152	0	2
-77.0	0	77	0	1
-78.0	0	78	0	1
-80.0	0	80	0	1
-82.0	0	82	0	1
-83.0	0	166	0	2
-84.0	0	168	0	2
-85.0	0	85	0	1
-86.0	0	86	0	1
-87.0	0	87	0	1
-90.0	0	270	0	3
-92.0	0	92	0	1
-95.0	0	190	0	2
-96.0	0	96	0	1
-97.0	0	194	0	2
-98.0	0	196	0	2
-100.0	0	200	0	2
-103.0	0	206	0	2
-104.0	0	208	0	2
-105.0	0	105	0	1
-111.0	0	111	0	1
-113.0	0	226	0	2
-114.0	0	114	0	1
-116.0	0	116	0	1
-118.0	0	236	0	2
-119.0	0	357	0	3
-120.0	0	240	0	2
-125.0	0	250	0	2
-126.0	0	126	0	1
-128.0	0	384	0	3
-129.0	0	258	0	2
-131.0	0	131	0	1
-133.0	0	133	0	1
-134.0	0	268	0	2
-136.0	0	136	0	1
-137.0	0	274	0	2
-138.0	0	552	0	4
-143.0	0	143	0	1
-145.0	0	145	0	1
-146.0	0	292	0	2
-149.0	0	298	0	2
-150.0	0	150	0	1
-152.0	0	304	0	2
-153.0	0	153	0	1
-155.0	0	155	0	1
-156.0	0	156	0	1
-157.0	0	157	0	1
-158.0	0	158	0	1
-160.0	0	160	0	1
-162.0	0	162	0	1
-163.0	0	163	0	1
-164.0	0	328	0	2
-165.0	0	330	0	2
-166.0	0	166	0	1
-167.0	0	501	0	3
-168.0	0	168	0	1
-169.0	0	676	0	4
-170.0	0	170	0	1
-172.0	0	344	0	2
-174.0	0	348	0	2
-175.0	0	350	0	2
-176.0	0	352	0	2
-177.0	0	177	0	1
-178.0	0	178	0	1
-179.0	0	358	0	2
-180.0	0	180	0	1
-181.0	0	181	0	1
-183.0	0	183	0	1
-186.0	0	186	0	1
-187.0	0	561	0	3
-189.0	0	189	0	1
-190.0	0	190	0	1
-191.0	0	382	0	2
-192.0	0	192	0	1
-193.0	0	579	0	3
-194.0	0	194	0	1
-195.0	0	390	0	2
-196.0	0	196	0	1
-197.0	0	394	0	2
-199.0	0	597	0	3
-200.0	0	400	0	2
-201.0	0	201	0	1
-202.0	0	202	0	1
-203.0	0	406	0	2
-205.0	0	410	0	2
-207.0	0	414	0	2
-208.0	0	624	0	3
-209.0	0	418	0	2
-213.0	0	426	0	2
-214.0	0	214	0	1
-216.0	0	432	0	2
-217.0	0	434	0	2
-218.0	0	218	0	1
-219.0	0	438	0	2
-221.0	0	442	0	2
-222.0	0	222	0	1
-223.0	0	446	0	2
-224.0	0	448	0	2
-226.0	0	226	0	1
-228.0	0	228	0	1
-229.0	0	458	0	2
-230.0	0	1150	0	5
-233.0	0	466	0	2
-235.0	0	235	0	1
-237.0	0	474	0	2
-238.0	0	476	0	2
-239.0	0	478	0	2
-241.0	0	241	0	1
-242.0	0	484	0	2
-244.0	0	244	0	1
-247.0	0	247	0	1
-248.0	0	248	0	1
-249.0	0	249	0	1
-252.0	0	252	0	1
-255.0	0	510	0	2
-256.0	0	512	0	2
-257.0	0	257	0	1
-258.0	0	258	0	1
-260.0	0	260	0	1
-262.0	0	262	0	1
-263.0	0	263	0	1
-265.0	0	530	0	2
-266.0	0	266	0	1
-272.0	0	544	0	2
-273.0	0	819	0	3
-274.0	0	274	0	1
-275.0	0	275	0	1
-277.0	0	1108	0	4
-278.0	0	556	0	2
-280.0	0	560	0	2
-281.0	0	562	0	2
-282.0	0	564	0	2
-283.0	0	283	0	1
-284.0	0	284	0	1
-285.0	0	285	0	1
-286.0	0	286	0	1
-287.0	0	287	0	1
-288.0	0	576	0	2
-289.0	0	289	0	1
-291.0	0	291	0	1
-292.0	0	292	0	1
-296.0	0	296	0	1
-298.0	0	894	0	3
-302.0	0	302	0	1
-305.0	0	305	0	1
-306.0	0	306	0	1
-307.0	0	614	0	2
-308.0	0	308	0	1
-309.0	0	618	0	2
-310.0	0	310	0	1
-311.0	0	933	0	3
-315.0	0	315	0	1
-316.0	0	948	0	3
-317.0	0	634	0	2
-318.0	0	954	0	3
-321.0	0	642	0	2
-322.0	0	644	0	2
-323.0	0	323	0	1
-325.0	0	650	0	2
-327.0	0	981	0	3
-331.0	0	662	0	2
-332.0	0	332	0	1
-333.0	0	666	0	2
-335.0	0	335	0	1
-336.0	0	336	0	1
-338.0	0	338	0	1
-339.0	0	339	0	1
-341.0	0	341	0	1
-342.0	0	684	0	2
-344.0	0	688	0	2
-345.0	0	345	0	1
-348.0	0	1740	0	5
-351.0	0	351	0	1
-353.0	0	706	0	2
-356.0	0	356	0	1
-360.0	0	360	0	1
-362.0	0	362	0	1
-364.0	0	364	0	1
-365.0	0	365	0	1
-366.0	0	366	0	1
-367.0	0	734	0	2
-368.0	0	368	0	1
-369.0	0	1107	0	3
-373.0	0	373	0	1
-374.0	0	374	0	1
-375.0	0	375	0	1
-377.0	0	377	0	1
-378.0	0	378	0	1
-379.0	0	379	0	1
-382.0	0	764	0	2
-384.0	0	1152	0	3
-386.0	0	386	0	1
-389.0	0	389	0	1
-392.0	0	392	0	1
-393.0	0	393	0	1
-394.0	0	394	0	1
-395.0	0	790	0	2
-396.0	0	1188	0	3
-397.0	0	794	0	2
-399.0	0	798	0	2
-400.0	0	400	0	1
-401.0	0	2005	0	5
-402.0	0	402	0	1
-403.0	0	1209	0	3
-404.0	0	808	0	2
-406.0	0	1624	0	4
-407.0	0	407	0	1
-409.0	0	1227	0	3
-411.0	0	411	0	1
-413.0	0	826	0	2
-414.0	0	828	0	2
-417.0	0	1251	0	3
-418.0	0	418	0	1
-419.0	0	419	0	1
-421.0	0	421	0	1
-424.0	0	848	0	2
-427.0	0	427	0	1
-429.0	0	858	0	2
-430.0	0	1290	0	3
-431.0	0	1293	0	3
-432.0	0	432	0	1
-435.0	0	435	0	1
-436.0	0	436	0	1
-437.0	0	437	0	1
-438.0	0	1314	0	3
-439.0	0	878	0	2
-443.0	0	443	0	1
-444.0	0	444	0	1
-446.0	0	446	0	1
-448.0	0	448	0	1
-449.0	0	449	0	1
-452.0	0	452	0	1
-453.0	0	453	0	1
-454.0	0	1362	0	3
-455.0	0	455	0	1
-457.0	0	457	0	1
-458.0	0	916	0	2
-459.0	0	918	0	2
-460.0	0	460	0	1
-462.0	0	924	0	2
-463.0	0	926	0	2
-466.0	0	1398	0	3
-467.0	0	467	0	1
-468.0	0	1872	0	4
-469.0	0	2345	0	5
-470.0	0	470	0	1
-472.0	0	472	0	1
-475.0	0	475	0	1
-477.0	0	477	0	1
-478.0	0	956	0	2
-479.0	0	479	0	1
-480.0	0	1440	0	3
-481.0	0	481	0	1
-482.0	0	482	0	1
-483.0	0	483	0	1
-484.0	0	484	0	1
-485.0	0	485	0	1
-487.0	0	487	0	1
-489.0	0	1956	0	4
-490.0	0	490	0	1
-491.0	0	491	0	1
-492.0	0	984	0	2
-493.0	0	493	0	1
-494.0	0	494	0	1
-495.0	0	495	0	1
-496.0	0	496	0	1
-497.0	0	497	0	1
-498.0	0	1494	0	3
diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c
deleted file mode 100644
index f23b45c32eccaf5c3071d78ed49ff05085d991bd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c	
+++ /dev/null
@@ -1,309 +0,0 @@
-0.0	0.0	0	0	0	3
-0.0	2.0	0	2	0	1
-0.0	4.0	0	4	0	1
-0.0	5.0	0	15	0	3
-0.0	8.0	0	8	0	1
-0.0	9.0	0	9	0	1
-0.0	10.0	0	10	0	1
-0.0	11.0	0	11	0	1
-0.0	12.0	0	24	0	2
-0.0	15.0	0	30	0	2
-0.0	17.0	0	17	0	1
-0.0	18.0	0	36	0	2
-0.0	19.0	0	19	0	1
-0.0	20.0	0	20	0	1
-0.0	24.0	0	48	0	2
-0.0	26.0	0	52	0	2
-0.0	27.0	0	27	0	1
-0.0	28.0	0	28	0	1
-0.0	30.0	0	30	0	1
-0.0	33.0	0	33	0	1
-0.0	34.0	0	34	0	1
-0.0	35.0	0	105	0	3
-0.0	37.0	0	74	0	2
-0.0	41.0	0	41	0	1
-0.0	42.0	0	84	0	2
-0.0	43.0	0	43	0	1
-0.0	44.0	0	44	0	1
-0.0	47.0	0	47	0	1
-0.0	51.0	0	102	0	2
-0.0	53.0	0	53	0	1
-0.0	54.0	0	54	0	1
-0.0	57.0	0	57	0	1
-0.0	58.0	0	116	0	2
-0.0	64.0	0	64	0	1
-0.0	65.0	0	65	0	1
-0.0	66.0	0	66	0	1
-0.0	67.0	0	134	0	2
-0.0	69.0	0	69	0	1
-0.0	70.0	0	210	0	3
-0.0	72.0	0	144	0	2
-0.0	74.0	0	74	0	1
-0.0	76.0	0	152	0	2
-0.0	77.0	0	77	0	1
-0.0	78.0	0	78	0	1
-0.0	80.0	0	80	0	1
-0.0	82.0	0	82	0	1
-0.0	83.0	0	166	0	2
-0.0	84.0	0	168	0	2
-0.0	85.0	0	85	0	1
-0.0	86.0	0	86	0	1
-0.0	87.0	0	87	0	1
-0.0	90.0	0	270	0	3
-0.0	92.0	0	92	0	1
-0.0	95.0	0	190	0	2
-0.0	96.0	0	96	0	1
-0.0	97.0	0	194	0	2
-0.0	98.0	0	196	0	2
-0.0	100.0	0	200	0	2
-0.0	103.0	0	206	0	2
-0.0	104.0	0	208	0	2
-0.0	105.0	0	105	0	1
-0.0	111.0	0	111	0	1
-0.0	113.0	0	226	0	2
-0.0	114.0	0	114	0	1
-0.0	116.0	0	116	0	1
-0.0	118.0	0	236	0	2
-0.0	119.0	0	357	0	3
-0.0	120.0	0	240	0	2
-0.0	125.0	0	250	0	2
-0.0	126.0	0	126	0	1
-0.0	128.0	0	384	0	3
-0.0	129.0	0	258	0	2
-0.0	131.0	0	131	0	1
-0.0	133.0	0	133	0	1
-0.0	134.0	0	268	0	2
-0.0	136.0	0	136	0	1
-0.0	137.0	0	274	0	2
-0.0	138.0	0	552	0	4
-0.0	143.0	0	143	0	1
-0.0	145.0	0	145	0	1
-0.0	146.0	0	292	0	2
-0.0	149.0	0	298	0	2
-0.0	150.0	0	150	0	1
-0.0	152.0	0	304	0	2
-0.0	153.0	0	153	0	1
-0.0	155.0	0	155	0	1
-0.0	156.0	0	156	0	1
-0.0	157.0	0	157	0	1
-0.0	158.0	0	158	0	1
-0.0	160.0	0	160	0	1
-0.0	162.0	0	162	0	1
-0.0	163.0	0	163	0	1
-0.0	164.0	0	328	0	2
-0.0	165.0	0	330	0	2
-0.0	166.0	0	166	0	1
-0.0	167.0	0	501	0	3
-0.0	168.0	0	168	0	1
-0.0	169.0	0	676	0	4
-0.0	170.0	0	170	0	1
-0.0	172.0	0	344	0	2
-0.0	174.0	0	348	0	2
-0.0	175.0	0	350	0	2
-0.0	176.0	0	352	0	2
-0.0	177.0	0	177	0	1
-0.0	178.0	0	178	0	1
-0.0	179.0	0	358	0	2
-0.0	180.0	0	180	0	1
-0.0	181.0	0	181	0	1
-0.0	183.0	0	183	0	1
-0.0	186.0	0	186	0	1
-0.0	187.0	0	561	0	3
-0.0	189.0	0	189	0	1
-0.0	190.0	0	190	0	1
-0.0	191.0	0	382	0	2
-0.0	192.0	0	192	0	1
-0.0	193.0	0	579	0	3
-0.0	194.0	0	194	0	1
-0.0	195.0	0	390	0	2
-0.0	196.0	0	196	0	1
-0.0	197.0	0	394	0	2
-0.0	199.0	0	597	0	3
-0.0	200.0	0	400	0	2
-0.0	201.0	0	201	0	1
-0.0	202.0	0	202	0	1
-0.0	203.0	0	406	0	2
-0.0	205.0	0	410	0	2
-0.0	207.0	0	414	0	2
-0.0	208.0	0	624	0	3
-0.0	209.0	0	418	0	2
-0.0	213.0	0	426	0	2
-0.0	214.0	0	214	0	1
-0.0	216.0	0	432	0	2
-0.0	217.0	0	434	0	2
-0.0	218.0	0	218	0	1
-0.0	219.0	0	438	0	2
-0.0	221.0	0	442	0	2
-0.0	222.0	0	222	0	1
-0.0	223.0	0	446	0	2
-0.0	224.0	0	448	0	2
-0.0	226.0	0	226	0	1
-0.0	228.0	0	228	0	1
-0.0	229.0	0	458	0	2
-0.0	230.0	0	1150	0	5
-0.0	233.0	0	466	0	2
-0.0	235.0	0	235	0	1
-0.0	237.0	0	474	0	2
-0.0	238.0	0	476	0	2
-0.0	239.0	0	478	0	2
-0.0	241.0	0	241	0	1
-0.0	242.0	0	484	0	2
-0.0	244.0	0	244	0	1
-0.0	247.0	0	247	0	1
-0.0	248.0	0	248	0	1
-0.0	249.0	0	249	0	1
-0.0	252.0	0	252	0	1
-0.0	255.0	0	510	0	2
-0.0	256.0	0	512	0	2
-0.0	257.0	0	257	0	1
-0.0	258.0	0	258	0	1
-0.0	260.0	0	260	0	1
-0.0	262.0	0	262	0	1
-0.0	263.0	0	263	0	1
-0.0	265.0	0	530	0	2
-0.0	266.0	0	266	0	1
-0.0	272.0	0	544	0	2
-0.0	273.0	0	819	0	3
-0.0	274.0	0	274	0	1
-0.0	275.0	0	275	0	1
-0.0	277.0	0	1108	0	4
-0.0	278.0	0	556	0	2
-0.0	280.0	0	560	0	2
-0.0	281.0	0	562	0	2
-0.0	282.0	0	564	0	2
-0.0	283.0	0	283	0	1
-0.0	284.0	0	284	0	1
-0.0	285.0	0	285	0	1
-0.0	286.0	0	286	0	1
-0.0	287.0	0	287	0	1
-0.0	288.0	0	576	0	2
-0.0	289.0	0	289	0	1
-0.0	291.0	0	291	0	1
-0.0	292.0	0	292	0	1
-0.0	296.0	0	296	0	1
-0.0	298.0	0	894	0	3
-0.0	302.0	0	302	0	1
-0.0	305.0	0	305	0	1
-0.0	306.0	0	306	0	1
-0.0	307.0	0	614	0	2
-0.0	308.0	0	308	0	1
-0.0	309.0	0	618	0	2
-0.0	310.0	0	310	0	1
-0.0	311.0	0	933	0	3
-0.0	315.0	0	315	0	1
-0.0	316.0	0	948	0	3
-0.0	317.0	0	634	0	2
-0.0	318.0	0	954	0	3
-0.0	321.0	0	642	0	2
-0.0	322.0	0	644	0	2
-0.0	323.0	0	323	0	1
-0.0	325.0	0	650	0	2
-0.0	327.0	0	981	0	3
-0.0	331.0	0	662	0	2
-0.0	332.0	0	332	0	1
-0.0	333.0	0	666	0	2
-0.0	335.0	0	335	0	1
-0.0	336.0	0	336	0	1
-0.0	338.0	0	338	0	1
-0.0	339.0	0	339	0	1
-0.0	341.0	0	341	0	1
-0.0	342.0	0	684	0	2
-0.0	344.0	0	688	0	2
-0.0	345.0	0	345	0	1
-0.0	348.0	0	1740	0	5
-0.0	351.0	0	351	0	1
-0.0	353.0	0	706	0	2
-0.0	356.0	0	356	0	1
-0.0	360.0	0	360	0	1
-0.0	362.0	0	362	0	1
-0.0	364.0	0	364	0	1
-0.0	365.0	0	365	0	1
-0.0	366.0	0	366	0	1
-0.0	367.0	0	734	0	2
-0.0	368.0	0	368	0	1
-0.0	369.0	0	1107	0	3
-0.0	373.0	0	373	0	1
-0.0	374.0	0	374	0	1
-0.0	375.0	0	375	0	1
-0.0	377.0	0	377	0	1
-0.0	378.0	0	378	0	1
-0.0	379.0	0	379	0	1
-0.0	382.0	0	764	0	2
-0.0	384.0	0	1152	0	3
-0.0	386.0	0	386	0	1
-0.0	389.0	0	389	0	1
-0.0	392.0	0	392	0	1
-0.0	393.0	0	393	0	1
-0.0	394.0	0	394	0	1
-0.0	395.0	0	790	0	2
-0.0	396.0	0	1188	0	3
-0.0	397.0	0	794	0	2
-0.0	399.0	0	798	0	2
-0.0	400.0	0	400	0	1
-0.0	401.0	0	2005	0	5
-0.0	402.0	0	402	0	1
-0.0	403.0	0	1209	0	3
-0.0	404.0	0	808	0	2
-0.0	406.0	0	1624	0	4
-0.0	407.0	0	407	0	1
-0.0	409.0	0	1227	0	3
-0.0	411.0	0	411	0	1
-0.0	413.0	0	826	0	2
-0.0	414.0	0	828	0	2
-0.0	417.0	0	1251	0	3
-0.0	418.0	0	418	0	1
-0.0	419.0	0	419	0	1
-0.0	421.0	0	421	0	1
-0.0	424.0	0	848	0	2
-0.0	427.0	0	427	0	1
-0.0	429.0	0	858	0	2
-0.0	430.0	0	1290	0	3
-0.0	431.0	0	1293	0	3
-0.0	432.0	0	432	0	1
-0.0	435.0	0	435	0	1
-0.0	436.0	0	436	0	1
-0.0	437.0	0	437	0	1
-0.0	438.0	0	1314	0	3
-0.0	439.0	0	878	0	2
-0.0	443.0	0	443	0	1
-0.0	444.0	0	444	0	1
-0.0	446.0	0	446	0	1
-0.0	448.0	0	448	0	1
-0.0	449.0	0	449	0	1
-0.0	452.0	0	452	0	1
-0.0	453.0	0	453	0	1
-0.0	454.0	0	1362	0	3
-0.0	455.0	0	455	0	1
-0.0	457.0	0	457	0	1
-0.0	458.0	0	916	0	2
-0.0	459.0	0	918	0	2
-0.0	460.0	0	460	0	1
-0.0	462.0	0	924	0	2
-0.0	463.0	0	926	0	2
-0.0	466.0	0	1398	0	3
-0.0	467.0	0	467	0	1
-0.0	468.0	0	1872	0	4
-0.0	469.0	0	2345	0	5
-0.0	470.0	0	470	0	1
-0.0	472.0	0	472	0	1
-0.0	475.0	0	475	0	1
-0.0	477.0	0	477	0	1
-0.0	478.0	0	956	0	2
-0.0	479.0	0	479	0	1
-0.0	480.0	0	1440	0	3
-0.0	481.0	0	481	0	1
-0.0	482.0	0	482	0	1
-0.0	483.0	0	483	0	1
-0.0	484.0	0	484	0	1
-0.0	485.0	0	485	0	1
-0.0	487.0	0	487	0	1
-0.0	489.0	0	1956	0	4
-0.0	490.0	0	490	0	1
-0.0	491.0	0	491	0	1
-0.0	492.0	0	984	0	2
-0.0	493.0	0	493	0	1
-0.0	494.0	0	494	0	1
-0.0	495.0	0	495	0	1
-0.0	496.0	0	496	0	1
-0.0	497.0	0	497	0	1
-0.0	498.0	0	1494	0	3
diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d
deleted file mode 100644
index 7839d714c25d23ed710712620c53cfc30de7548b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d	
+++ /dev/null
@@ -1,309 +0,0 @@
-0	3
-0	1
-0	1
-0	3
-0	1
-0	1
-0	1
-0	1
-0	2
-0	2
-0	1
-0	2
-0	1
-0	1
-0	2
-0	2
-0	1
-0	1
-0	1
-0	1
-0	1
-0	3
-0	2
-0	1
-0	2
-0	1
-0	1
-0	1
-0	2
-0	1
-0	1
-0	1
-0	2
-0	1
-0	1
-0	1
-0	2
-0	1
-0	3
-0	2
-0	1
-0	2
-0	1
-0	1
-0	1
-0	1
-0	2
-0	2
-0	1
-0	1
-0	1
-0	3
-0	1
-0	2
-0	1
-0	2
-0	2
-0	2
-0	2
-0	2
-0	1
-0	1
-0	2
-0	1
-0	1
-0	2
-0	3
-0	2
-0	2
-0	1
-0	3
-0	2
-0	1
-0	1
-0	2
-0	1
-0	2
-0	4
-0	1
-0	1
-0	2
-0	2
-0	1
-0	2
-0	1
-0	1
-0	1
-0	1
-0	1
-0	1
-0	1
-0	1
-0	2
-0	2
-0	1
-0	3
-0	1
-0	4
-0	1
-0	2
-0	2
-0	2
-0	2
-0	1
-0	1
-0	2
-0	1
-0	1
-0	1
-0	1
-0	3
-0	1
-0	1
-0	2
-0	1
-0	3
-0	1
-0	2
-0	1
-0	2
-0	3
-0	2
-0	1
-0	1
-0	2
-0	2
-0	2
-0	3
-0	2
-0	2
-0	1
-0	2
-0	2
-0	1
-0	2
-0	2
-0	1
-0	2
-0	2
-0	1
-0	1
-0	2
-0	5
-0	2
-0	1
-0	2
-0	2
-0	2
-0	1
-0	2
-0	1
-0	1
-0	1
-0	1
-0	1
-0	2
-0	2
-0	1
-0	1
-0	1
-0	1
-0	1
-0	2
-0	1
-0	2
-0	3
-0	1
-0	1
-0	4
-0	2
-0	2
-0	2
-0	2
-0	1
-0	1
-0	1
-0	1
-0	1
-0	2
-0	1
-0	1
-0	1
-0	1
-0	3
-0	1
-0	1
-0	1
-0	2
-0	1
-0	2
-0	1
-0	3
-0	1
-0	3
-0	2
-0	3
-0	2
-0	2
-0	1
-0	2
-0	3
-0	2
-0	1
-0	2
-0	1
-0	1
-0	1
-0	1
-0	1
-0	2
-0	2
-0	1
-0	5
-0	1
-0	2
-0	1
-0	1
-0	1
-0	1
-0	1
-0	1
-0	2
-0	1
-0	3
-0	1
-0	1
-0	1
-0	1
-0	1
-0	1
-0	2
-0	3
-0	1
-0	1
-0	1
-0	1
-0	1
-0	2
-0	3
-0	2
-0	2
-0	1
-0	5
-0	1
-0	3
-0	2
-0	4
-0	1
-0	3
-0	1
-0	2
-0	2
-0	3
-0	1
-0	1
-0	1
-0	2
-0	1
-0	2
-0	3
-0	3
-0	1
-0	1
-0	1
-0	1
-0	3
-0	2
-0	1
-0	1
-0	1
-0	1
-0	1
-0	1
-0	1
-0	3
-0	1
-0	1
-0	2
-0	2
-0	1
-0	2
-0	2
-0	3
-0	1
-0	4
-0	5
-0	1
-0	1
-0	1
-0	1
-0	2
-0	1
-0	3
-0	1
-0	1
-0	1
-0	1
-0	1
-0	1
-0	4
-0	1
-0	1
-0	2
-0	1
-0	1
-0	1
-0	1
-0	1
-0	3
diff --git a/sql/hive/src/test/resources/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286 b/sql/hive/src/test/resources/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286
index bbe37f8e2a790434f1100dc8de1a96cf6b8d0558..2899c62e08afd0d42f3846b877aedcbd7ec564b7 100644
--- a/sql/hive/src/test/resources/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286	
+++ b/sql/hive/src/test/resources/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286	
@@ -1 +1 @@
-86	\	\
\ No newline at end of file
+86	\	\
diff --git a/sql/hive/src/test/resources/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 b/sql/hive/src/test/resources/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672
index 8007988316af0e632d8973f4ddf30f427559005f..16c12c524e7d27972b7e85344d1f20f191339e80 100644
--- a/sql/hive/src/test/resources/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672
+++ b/sql/hive/src/test/resources/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672
@@ -172,4 +172,4 @@
 194	val_194
 414	val_414
 126	val_126
-169	val_169
\ No newline at end of file
+169	val_169
diff --git a/sql/hive/src/test/resources/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7 b/sql/hive/src/test/resources/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7
index 595d7af04a69a3caced5b0c697589eda9bc388ed..1cce52daebce91cbd315a6b20686d37a75f7383b 100644
--- a/sql/hive/src/test/resources/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7	
+++ b/sql/hive/src/test/resources/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7	
@@ -306,4 +306,4 @@
 96.0	val_95
 97.0	val_96
 98.0	val_97
-99.0	val_98
\ No newline at end of file
+99.0	val_98
diff --git a/sql/hive/src/test/resources/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 b/sql/hive/src/test/resources/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935
index 3d2dbbd73154361316a8b95bfaf1b439af54d78d..7f2be0e640110d7fd649302ce5ef07c9e41195d4 100644
--- a/sql/hive/src/test/resources/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935	
+++ b/sql/hive/src/test/resources/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935	
@@ -1 +1 @@
-261.182
\ No newline at end of file
+261.182
diff --git a/sql/hive/src/test/resources/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5 b/sql/hive/src/test/resources/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5
index cbc70c89f88597895065222f3ec07de0dacf607d..ed2b1a0e7c53a6e2e1230fae7f778d25768977fc 100644
--- a/sql/hive/src/test/resources/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5	
+++ b/sql/hive/src/test/resources/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5	
@@ -1 +1 @@
-260.182
\ No newline at end of file
+260.182
diff --git a/sql/hive/src/test/resources/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c
index ecafeaea5f61ab99d238c32150ce31e27145f46c..85c1918f46567d9bb13c443b5142296d0d2fbc14 100644
--- a/sql/hive/src/test/resources/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c
@@ -1,2 +1,2 @@
 src
-srcpart
\ No newline at end of file
+srcpart
diff --git a/sql/hive/src/test/resources/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b
index f1ea8b0260d2cd69dd67e1427ff084ecc280a616..0fd332ea35b85c962cb12e6f2ddb9a6034be9572 100644
--- a/sql/hive/src/test/resources/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b
+++ b/sql/hive/src/test/resources/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b
@@ -1 +1 @@
-ds=2010-01-01
\ No newline at end of file
+ds=2010-01-01
diff --git a/sql/hive/src/test/resources/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b
index f1ea8b0260d2cd69dd67e1427ff084ecc280a616..0fd332ea35b85c962cb12e6f2ddb9a6034be9572 100644
--- a/sql/hive/src/test/resources/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b
+++ b/sql/hive/src/test/resources/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b
@@ -1 +1 @@
-ds=2010-01-01
\ No newline at end of file
+ds=2010-01-01
diff --git a/sql/hive/src/test/resources/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b
index 121aebdfab9afd56af42e98c65a3924fae0766dd..2c7bbaac173188628efdbd4270b4ad7a8beeb934 100644
--- a/sql/hive/src/test/resources/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b
+++ b/sql/hive/src/test/resources/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b
@@ -1,2 +1,2 @@
 ds=2010-01-01
-ds=2010-01-02
\ No newline at end of file
+ds=2010-01-02
diff --git a/sql/hive/src/test/resources/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b
index 64467c12563c831e35f8c20eaa93ced9feeab766..7d9d4194c5cd13cecd67d750e972764eb3685e41 100644
--- a/sql/hive/src/test/resources/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b
+++ b/sql/hive/src/test/resources/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b
@@ -1,3 +1,3 @@
 ds=2010-01-01
 ds=2010-01-02
-ds=2010-01-03
\ No newline at end of file
+ds=2010-01-03
diff --git a/sql/hive/src/test/resources/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b
index f1ea8b0260d2cd69dd67e1427ff084ecc280a616..0fd332ea35b85c962cb12e6f2ddb9a6034be9572 100644
--- a/sql/hive/src/test/resources/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b
+++ b/sql/hive/src/test/resources/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b
@@ -1 +1 @@
-ds=2010-01-01
\ No newline at end of file
+ds=2010-01-01
diff --git a/sql/hive/src/test/resources/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b
index f1ea8b0260d2cd69dd67e1427ff084ecc280a616..0fd332ea35b85c962cb12e6f2ddb9a6034be9572 100644
--- a/sql/hive/src/test/resources/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b
+++ b/sql/hive/src/test/resources/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b
@@ -1 +1 @@
-ds=2010-01-01
\ No newline at end of file
+ds=2010-01-01
diff --git a/sql/hive/src/test/resources/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b
index 121aebdfab9afd56af42e98c65a3924fae0766dd..2c7bbaac173188628efdbd4270b4ad7a8beeb934 100644
--- a/sql/hive/src/test/resources/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b
+++ b/sql/hive/src/test/resources/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b
@@ -1,2 +1,2 @@
 ds=2010-01-01
-ds=2010-01-02
\ No newline at end of file
+ds=2010-01-02
diff --git a/sql/hive/src/test/resources/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b
index 64467c12563c831e35f8c20eaa93ced9feeab766..7d9d4194c5cd13cecd67d750e972764eb3685e41 100644
--- a/sql/hive/src/test/resources/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b
+++ b/sql/hive/src/test/resources/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b
@@ -1,3 +1,3 @@
 ds=2010-01-01
 ds=2010-01-02
-ds=2010-01-03
\ No newline at end of file
+ds=2010-01-03
diff --git a/sql/hive/src/test/resources/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1 b/sql/hive/src/test/resources/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1
index da21345385b2aa27d2d96e3e8139e2afb9aaa4bb..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 100644
--- a/sql/hive/src/test/resources/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1
+++ b/sql/hive/src/test/resources/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1
@@ -1,29 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ALTERTABLE_ADDPARTS add_part_test TOK_IFNOTEXISTS (TOK_PARTSPEC (TOK_PARTVAL ds '2010-01-01')) (TOK_PARTITIONLOCATION 'A') (TOK_PARTSPEC (TOK_PARTVAL ds '2010-02-01')) (TOK_PARTITIONLOCATION 'B') (TOK_PARTSPEC (TOK_PARTVAL ds '2010-03-01')) (TOK_PARTSPEC (TOK_PARTVAL ds '2010-04-01')) (TOK_PARTITIONLOCATION 'C'))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-  Stage-2 is a root stage
-  Stage-3 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Add Partition Operator:
-          Location: A
-          Spec: {ds=2010-01-01}
-
-  Stage: Stage-1
-      Add Partition Operator:
-          Location: B
-          Spec: {ds=2010-02-01}
-
-  Stage: Stage-2
-      Add Partition Operator:
-          Spec: {ds=2010-03-01}
-
-  Stage: Stage-3
-      Add Partition Operator:
-          Location: C
-          Spec: {ds=2010-04-01}
-
diff --git a/sql/hive/src/test/resources/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c b/sql/hive/src/test/resources/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c
index fc4021c39737f7a1eceb56938eea56e98edac492..0587978b3efe91b58b3361bb1783d3126c8a98cb 100644
--- a/sql/hive/src/test/resources/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c
+++ b/sql/hive/src/test/resources/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c
@@ -1,4 +1,4 @@
 100	100	2010-01-01
 200	200	2010-02-01
 400	300	2010-03-01
-500	400	2010-04-01
\ No newline at end of file
+500	400	2010-04-01
diff --git a/sql/hive/src/test/resources/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304 b/sql/hive/src/test/resources/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304
+++ b/sql/hive/src/test/resources/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 b/sql/hive/src/test/resources/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196
+++ b/sql/hive/src/test/resources/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 b/sql/hive/src/test/resources/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8
+++ b/sql/hive/src/test/resources/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591
index 62a8ea1d115fb76b2c1f6174716af3752be99fb1..4e4f6dc8ef5a7cda414e8ae5458ec22d49f4399d 100644
--- a/sql/hive/src/test/resources/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591
+++ b/sql/hive/src/test/resources/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591
@@ -1,10 +1,10 @@
-a                   	int                 	None                
-b                   	int                 	None                
-insertdate          	string              	None                
+a                   	int                 	                    
+b                   	int                 	                    
+insertdate          	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-insertdate          	string              	None                
+insertdate          	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1389728471, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728471}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1413871628, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1413871628}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591
index 72621add45bb39b81c85e98f7b9bbbf70e41caaf..3633169d9d5f2ed4a01961ab0c2ad629802672aa 100644
--- a/sql/hive/src/test/resources/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591
+++ b/sql/hive/src/test/resources/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591
@@ -1,10 +1,10 @@
-a                   	int                 	None                
-b                   	int                 	None                
-insertdate          	string              	None                
+a                   	int                 	                    
+b                   	int                 	                    
+insertdate          	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-insertdate          	string              	None                
+insertdate          	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1389728473, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1389728473}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1413871628, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1413871628}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591
index 72621add45bb39b81c85e98f7b9bbbf70e41caaf..3633169d9d5f2ed4a01961ab0c2ad629802672aa 100644
--- a/sql/hive/src/test/resources/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591
+++ b/sql/hive/src/test/resources/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591
@@ -1,10 +1,10 @@
-a                   	int                 	None                
-b                   	int                 	None                
-insertdate          	string              	None                
+a                   	int                 	                    
+b                   	int                 	                    
+insertdate          	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-insertdate          	string              	None                
+insertdate          	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1389728473, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1389728473}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1413871628, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1413871628}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter2-15-75a213649242c2410ea6846f08c91d75 b/sql/hive/src/test/resources/golden/alter2-15-75a213649242c2410ea6846f08c91d75
index 15e6bc8823150147b8eaffa2317e8b5cc352bc64..30f78a2dc1fb19a267063e2b768f7efbac9f5daa 100644
--- a/sql/hive/src/test/resources/golden/alter2-15-75a213649242c2410ea6846f08c91d75
+++ b/sql/hive/src/test/resources/golden/alter2-15-75a213649242c2410ea6846f08c91d75
@@ -1 +1 @@
-insertdate=2008-01-01
\ No newline at end of file
+insertdate=2008-01-01
diff --git a/sql/hive/src/test/resources/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591
index 72621add45bb39b81c85e98f7b9bbbf70e41caaf..3633169d9d5f2ed4a01961ab0c2ad629802672aa 100644
--- a/sql/hive/src/test/resources/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591
+++ b/sql/hive/src/test/resources/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591
@@ -1,10 +1,10 @@
-a                   	int                 	None                
-b                   	int                 	None                
-insertdate          	string              	None                
+a                   	int                 	                    
+b                   	int                 	                    
+insertdate          	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-insertdate          	string              	None                
+insertdate          	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1389728473, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1389728473}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1413871628, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1413871628}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter2-18-75a213649242c2410ea6846f08c91d75 b/sql/hive/src/test/resources/golden/alter2-18-75a213649242c2410ea6846f08c91d75
index d31318a383fc10c8bff53b834c59d7131d511f66..81b4ae118d12dd34203178520096e6fce8c16913 100644
--- a/sql/hive/src/test/resources/golden/alter2-18-75a213649242c2410ea6846f08c91d75
+++ b/sql/hive/src/test/resources/golden/alter2-18-75a213649242c2410ea6846f08c91d75
@@ -1,2 +1,2 @@
 insertdate=2008-01-01
-insertdate=2008-01-02
\ No newline at end of file
+insertdate=2008-01-02
diff --git a/sql/hive/src/test/resources/golden/alter2-20-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter2-20-9c36cac1372650b703400c60dd29042c
index ecafeaea5f61ab99d238c32150ce31e27145f46c..85c1918f46567d9bb13c443b5142296d0d2fbc14 100644
--- a/sql/hive/src/test/resources/golden/alter2-20-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/alter2-20-9c36cac1372650b703400c60dd29042c
@@ -1,2 +1,2 @@
 src
-srcpart
\ No newline at end of file
+srcpart
diff --git a/sql/hive/src/test/resources/golden/alter2-25-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-25-4ef75e12575453225738ea167c4617e5
index 83184977e9da8ebd3d56fae66156d54888490023..ea564fc70087acfd5e9a3b06068404a247d2e683 100644
--- a/sql/hive/src/test/resources/golden/alter2-25-4ef75e12575453225738ea167c4617e5
+++ b/sql/hive/src/test/resources/golden/alter2-25-4ef75e12575453225738ea167c4617e5
@@ -1,10 +1,10 @@
-a                   	int                 	None                
-b                   	int                 	None                
-insertdate          	string              	None                
+a                   	int                 	                    
+b                   	int                 	                    
+insertdate          	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-insertdate          	string              	None                
+insertdate          	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1389728475, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728475}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1413871629, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1413871629}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter2-28-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-28-4ef75e12575453225738ea167c4617e5
index 83184977e9da8ebd3d56fae66156d54888490023..ea564fc70087acfd5e9a3b06068404a247d2e683 100644
--- a/sql/hive/src/test/resources/golden/alter2-28-4ef75e12575453225738ea167c4617e5
+++ b/sql/hive/src/test/resources/golden/alter2-28-4ef75e12575453225738ea167c4617e5
@@ -1,10 +1,10 @@
-a                   	int                 	None                
-b                   	int                 	None                
-insertdate          	string              	None                
+a                   	int                 	                    
+b                   	int                 	                    
+insertdate          	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-insertdate          	string              	None                
+insertdate          	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1389728475, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728475}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1413871629, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1413871629}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter2-29-63f13c364546ddce5d2176c6604a948f b/sql/hive/src/test/resources/golden/alter2-29-63f13c364546ddce5d2176c6604a948f
index 15e6bc8823150147b8eaffa2317e8b5cc352bc64..30f78a2dc1fb19a267063e2b768f7efbac9f5daa 100644
--- a/sql/hive/src/test/resources/golden/alter2-29-63f13c364546ddce5d2176c6604a948f
+++ b/sql/hive/src/test/resources/golden/alter2-29-63f13c364546ddce5d2176c6604a948f
@@ -1 +1 @@
-insertdate=2008-01-01
\ No newline at end of file
+insertdate=2008-01-01
diff --git a/sql/hive/src/test/resources/golden/alter2-31-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-31-4ef75e12575453225738ea167c4617e5
index 83184977e9da8ebd3d56fae66156d54888490023..ea564fc70087acfd5e9a3b06068404a247d2e683 100644
--- a/sql/hive/src/test/resources/golden/alter2-31-4ef75e12575453225738ea167c4617e5
+++ b/sql/hive/src/test/resources/golden/alter2-31-4ef75e12575453225738ea167c4617e5
@@ -1,10 +1,10 @@
-a                   	int                 	None                
-b                   	int                 	None                
-insertdate          	string              	None                
+a                   	int                 	                    
+b                   	int                 	                    
+insertdate          	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-insertdate          	string              	None                
+insertdate          	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1389728475, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728475}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1413871629, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1413871629}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter2-32-63f13c364546ddce5d2176c6604a948f b/sql/hive/src/test/resources/golden/alter2-32-63f13c364546ddce5d2176c6604a948f
index d31318a383fc10c8bff53b834c59d7131d511f66..81b4ae118d12dd34203178520096e6fce8c16913 100644
--- a/sql/hive/src/test/resources/golden/alter2-32-63f13c364546ddce5d2176c6604a948f
+++ b/sql/hive/src/test/resources/golden/alter2-32-63f13c364546ddce5d2176c6604a948f
@@ -1,2 +1,2 @@
 insertdate=2008-01-01
-insertdate=2008-01-02
\ No newline at end of file
+insertdate=2008-01-02
diff --git a/sql/hive/src/test/resources/golden/alter2-35-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-35-4ef75e12575453225738ea167c4617e5
index 6e30936a881bddcde85a0d56a0700bf1a5602277..808dc578804ee62f44ca050b09d3717892ea3393 100644
--- a/sql/hive/src/test/resources/golden/alter2-35-4ef75e12575453225738ea167c4617e5
+++ b/sql/hive/src/test/resources/golden/alter2-35-4ef75e12575453225738ea167c4617e5
@@ -1,10 +1,10 @@
-a                   	int                 	None                
-b                   	int                 	None                
-insertdate          	string              	None                
+a                   	int                 	                    
+b                   	int                 	                    
+insertdate          	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-insertdate          	string              	None                
+insertdate          	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1389728477, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1389728477}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1413871629, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1413871629}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter2-38-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-38-4ef75e12575453225738ea167c4617e5
index 6e30936a881bddcde85a0d56a0700bf1a5602277..808dc578804ee62f44ca050b09d3717892ea3393 100644
--- a/sql/hive/src/test/resources/golden/alter2-38-4ef75e12575453225738ea167c4617e5
+++ b/sql/hive/src/test/resources/golden/alter2-38-4ef75e12575453225738ea167c4617e5
@@ -1,10 +1,10 @@
-a                   	int                 	None                
-b                   	int                 	None                
-insertdate          	string              	None                
+a                   	int                 	                    
+b                   	int                 	                    
+insertdate          	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-insertdate          	string              	None                
+insertdate          	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1389728477, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1389728477}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1413871629, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1413871629}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter2-39-63f13c364546ddce5d2176c6604a948f b/sql/hive/src/test/resources/golden/alter2-39-63f13c364546ddce5d2176c6604a948f
index 15e6bc8823150147b8eaffa2317e8b5cc352bc64..30f78a2dc1fb19a267063e2b768f7efbac9f5daa 100644
--- a/sql/hive/src/test/resources/golden/alter2-39-63f13c364546ddce5d2176c6604a948f
+++ b/sql/hive/src/test/resources/golden/alter2-39-63f13c364546ddce5d2176c6604a948f
@@ -1 +1 @@
-insertdate=2008-01-01
\ No newline at end of file
+insertdate=2008-01-01
diff --git a/sql/hive/src/test/resources/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591
index 62a8ea1d115fb76b2c1f6174716af3752be99fb1..4e4f6dc8ef5a7cda414e8ae5458ec22d49f4399d 100644
--- a/sql/hive/src/test/resources/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591
+++ b/sql/hive/src/test/resources/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591
@@ -1,10 +1,10 @@
-a                   	int                 	None                
-b                   	int                 	None                
-insertdate          	string              	None                
+a                   	int                 	                    
+b                   	int                 	                    
+insertdate          	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-insertdate          	string              	None                
+insertdate          	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1389728471, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728471}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1413871628, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1413871628}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter2-41-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-41-4ef75e12575453225738ea167c4617e5
index 6e30936a881bddcde85a0d56a0700bf1a5602277..808dc578804ee62f44ca050b09d3717892ea3393 100644
--- a/sql/hive/src/test/resources/golden/alter2-41-4ef75e12575453225738ea167c4617e5
+++ b/sql/hive/src/test/resources/golden/alter2-41-4ef75e12575453225738ea167c4617e5
@@ -1,10 +1,10 @@
-a                   	int                 	None                
-b                   	int                 	None                
-insertdate          	string              	None                
+a                   	int                 	                    
+b                   	int                 	                    
+insertdate          	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-insertdate          	string              	None                
+insertdate          	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1389728477, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1389728477}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter2, dbName:alter2_db, owner:marmbrus, createTime:1413871629, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter2_db.db/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{EXTERNAL=TRUE, transient_lastDdlTime=1413871629}, viewOriginalText:null, viewExpandedText:null, tableType:EXTERNAL_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter2-42-63f13c364546ddce5d2176c6604a948f b/sql/hive/src/test/resources/golden/alter2-42-63f13c364546ddce5d2176c6604a948f
index d31318a383fc10c8bff53b834c59d7131d511f66..81b4ae118d12dd34203178520096e6fce8c16913 100644
--- a/sql/hive/src/test/resources/golden/alter2-42-63f13c364546ddce5d2176c6604a948f
+++ b/sql/hive/src/test/resources/golden/alter2-42-63f13c364546ddce5d2176c6604a948f
@@ -1,2 +1,2 @@
 insertdate=2008-01-01
-insertdate=2008-01-02
\ No newline at end of file
+insertdate=2008-01-02
diff --git a/sql/hive/src/test/resources/golden/alter2-5-75a213649242c2410ea6846f08c91d75 b/sql/hive/src/test/resources/golden/alter2-5-75a213649242c2410ea6846f08c91d75
index 15e6bc8823150147b8eaffa2317e8b5cc352bc64..30f78a2dc1fb19a267063e2b768f7efbac9f5daa 100644
--- a/sql/hive/src/test/resources/golden/alter2-5-75a213649242c2410ea6846f08c91d75
+++ b/sql/hive/src/test/resources/golden/alter2-5-75a213649242c2410ea6846f08c91d75
@@ -1 +1 @@
-insertdate=2008-01-01
\ No newline at end of file
+insertdate=2008-01-01
diff --git a/sql/hive/src/test/resources/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591
index 62a8ea1d115fb76b2c1f6174716af3752be99fb1..4e4f6dc8ef5a7cda414e8ae5458ec22d49f4399d 100644
--- a/sql/hive/src/test/resources/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591
+++ b/sql/hive/src/test/resources/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591
@@ -1,10 +1,10 @@
-a                   	int                 	None                
-b                   	int                 	None                
-insertdate          	string              	None                
+a                   	int                 	                    
+b                   	int                 	                    
+insertdate          	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-insertdate          	string              	None                
+insertdate          	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1389728471, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728471}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter2, dbName:default, owner:marmbrus, createTime:1413871628, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null), FieldSchema(name:b, type:int, comment:null), FieldSchema(name:insertdate, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:insertdate, type:string, comment:null)], parameters:{transient_lastDdlTime=1413871628}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter2-8-75a213649242c2410ea6846f08c91d75 b/sql/hive/src/test/resources/golden/alter2-8-75a213649242c2410ea6846f08c91d75
index d31318a383fc10c8bff53b834c59d7131d511f66..81b4ae118d12dd34203178520096e6fce8c16913 100644
--- a/sql/hive/src/test/resources/golden/alter2-8-75a213649242c2410ea6846f08c91d75
+++ b/sql/hive/src/test/resources/golden/alter2-8-75a213649242c2410ea6846f08c91d75
@@ -1,2 +1,2 @@
 insertdate=2008-01-01
-insertdate=2008-01-02
\ No newline at end of file
+insertdate=2008-01-02
diff --git a/sql/hive/src/test/resources/golden/alter5-1-b8349afaf8e62dc6608a889c04ee3d4b b/sql/hive/src/test/resources/golden/alter3-1-47f70392b97b94cdb425b25bde204f58
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter5-1-b8349afaf8e62dc6608a889c04ee3d4b
rename to sql/hive/src/test/resources/golden/alter3-1-47f70392b97b94cdb425b25bde204f58
diff --git a/sql/hive/src/test/resources/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506 b/sql/hive/src/test/resources/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506
index ba1746da5ce697cd478e438c705296f367f8cf99..424f827129d716746c8364e8face922dec9b9458 100644
--- a/sql/hive/src/test/resources/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506
+++ b/sql/hive/src/test/resources/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506
@@ -1,11 +1,11 @@
-col1                	string              	None                
-pcol1               	string              	None                
-pcol2               	string              	None                
+col1                	string              	                    
+pcol1               	string              	                    
+pcol2               	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-pcol1               	string              	None                
-pcol2               	string              	None                
+pcol1               	string              	                    
+pcol2               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter3_like_renamed, dbName:default, owner:marmbrus, createTime:1389728495, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter3_like_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, last_modified_by=marmbrus, last_modified_time=1389728524, transient_lastDdlTime=1389728524, numRows=6, totalSize=171, rawDataSize=6}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter3_like_renamed, dbName:default, owner:marmbrus, createTime:1413871632, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter3_like_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413871646, transient_lastDdlTime=1413871646}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter3-16-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter3-16-9c36cac1372650b703400c60dd29042c
index ecafeaea5f61ab99d238c32150ce31e27145f46c..85c1918f46567d9bb13c443b5142296d0d2fbc14 100644
--- a/sql/hive/src/test/resources/golden/alter3-16-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/alter3-16-9c36cac1372650b703400c60dd29042c
@@ -1,2 +1,2 @@
 src
-srcpart
\ No newline at end of file
+srcpart
diff --git a/sql/hive/src/test/resources/golden/alter5-15-b8349afaf8e62dc6608a889c04ee3d4b b/sql/hive/src/test/resources/golden/alter3-21-91e32b3028ecc352dad8884829148311
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter5-15-b8349afaf8e62dc6608a889c04ee3d4b
rename to sql/hive/src/test/resources/golden/alter3-21-91e32b3028ecc352dad8884829148311
diff --git a/sql/hive/src/test/resources/golden/alter3-25-568a59760e5d3241b63d65cce595face b/sql/hive/src/test/resources/golden/alter3-25-568a59760e5d3241b63d65cce595face
index 8e609740f34a7f335bcbc34c865e04730bca5898..d1f5bd707fcb47cf84cd8d6d75b93f6e8ad86ece 100644
--- a/sql/hive/src/test/resources/golden/alter3-25-568a59760e5d3241b63d65cce595face
+++ b/sql/hive/src/test/resources/golden/alter3-25-568a59760e5d3241b63d65cce595face
@@ -3,4 +3,4 @@
 3	test_part:	test_part:
 4	test_part:	test_part:
 5	test_part:	test_part:
-6	test_part:	test_part:
\ No newline at end of file
+6	test_part:	test_part:
diff --git a/sql/hive/src/test/resources/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87 b/sql/hive/src/test/resources/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87
index 0e12e0e1663363a3bce42b98efa8a860f57499e0..ba284b7d4300abf9a444ae4b63f1e05ea1790237 100644
--- a/sql/hive/src/test/resources/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87
+++ b/sql/hive/src/test/resources/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87
@@ -1,11 +1,11 @@
-col1                	string              	None                
-pcol1               	string              	None                
-pcol2               	string              	None                
+col1                	string              	                    
+pcol1               	string              	                    
+pcol2               	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-pcol1               	string              	None                
-pcol2               	string              	None                
+pcol1               	string              	                    
+pcol2               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter3_renamed, dbName:alter3_db, owner:marmbrus, createTime:1389728526, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter3_db.db/alter3_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, last_modified_by=marmbrus, last_modified_time=1389728544, transient_lastDdlTime=1389728544, numRows=6, totalSize=171, rawDataSize=6}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter3_renamed, dbName:alter3_db, owner:marmbrus, createTime:1413871647, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter3_db.db/alter3_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413871654, transient_lastDdlTime=1413871654}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773 b/sql/hive/src/test/resources/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773
index 4acb920388cebd1a6c3b3790817c53f8c60a7271..ffd81df5a7bf4bcf3825c615ba138749c6db9125 100644
--- a/sql/hive/src/test/resources/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773
+++ b/sql/hive/src/test/resources/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773
@@ -1,11 +1,11 @@
-col1                	string              	None                
-pcol1               	string              	None                
-pcol2               	string              	None                
+col1                	string              	                    
+pcol1               	string              	                    
+pcol2               	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-pcol1               	string              	None                
-pcol2               	string              	None                
+pcol1               	string              	                    
+pcol2               	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[test_part:, test_part:], dbName:alter3_db, tableName:alter3_renamed, createTime:1389728543, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter3_db.db/alter3_renamed/pcol1=test_part%3A/pcol2=test_part%3A, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1389728544, numRows=6, totalSize=171, rawDataSize=6})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[test_part:, test_part:], dbName:alter3_db, tableName:alter3_renamed, createTime:1413871654, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter3_db.db/alter3_renamed/pcol1=test_part%3A/pcol2=test_part%3A, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413871654, COLUMN_STATS_ACCURATE=true, totalSize=171, numRows=6, rawDataSize=6})	
diff --git a/sql/hive/src/test/resources/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c b/sql/hive/src/test/resources/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c
index 8e609740f34a7f335bcbc34c865e04730bca5898..d1f5bd707fcb47cf84cd8d6d75b93f6e8ad86ece 100644
--- a/sql/hive/src/test/resources/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c
+++ b/sql/hive/src/test/resources/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c
@@ -3,4 +3,4 @@
 3	test_part:	test_part:
 4	test_part:	test_part:
 5	test_part:	test_part:
-6	test_part:	test_part:
\ No newline at end of file
+6	test_part:	test_part:
diff --git a/sql/hive/src/test/resources/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65 b/sql/hive/src/test/resources/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65
index 12b038204bef4f9ae6bfd2f97d5d837fbf6eeb79..5f38c1fce1523dcd740b2e9ece5246aa125716ac 100644
--- a/sql/hive/src/test/resources/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65
+++ b/sql/hive/src/test/resources/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65
@@ -1,11 +1,11 @@
-col1                	string              	None                
-pcol1               	string              	None                
-pcol2               	string              	None                
+col1                	string              	                    
+pcol1               	string              	                    
+pcol2               	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-pcol1               	string              	None                
-pcol2               	string              	None                
+pcol1               	string              	                    
+pcol2               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter3_like_renamed, dbName:alter3_db, owner:marmbrus, createTime:1389728527, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter3_db.db/alter3_like_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, last_modified_by=marmbrus, last_modified_time=1389728560, transient_lastDdlTime=1389728560, numRows=6, totalSize=171, rawDataSize=6}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter3_like_renamed, dbName:alter3_db, owner:marmbrus, createTime:1413871647, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter3_db.db/alter3_like_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413871661, transient_lastDdlTime=1413871661}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97 b/sql/hive/src/test/resources/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97
index 8e609740f34a7f335bcbc34c865e04730bca5898..d1f5bd707fcb47cf84cd8d6d75b93f6e8ad86ece 100644
--- a/sql/hive/src/test/resources/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97
+++ b/sql/hive/src/test/resources/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97
@@ -3,4 +3,4 @@
 3	test_part:	test_part:
 4	test_part:	test_part:
 5	test_part:	test_part:
-6	test_part:	test_part:
\ No newline at end of file
+6	test_part:	test_part:
diff --git a/sql/hive/src/test/resources/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 b/sql/hive/src/test/resources/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8
index 0c47fb0ac64aaaba206593c4db032f028a93c7b7..f4a4878f5ac17c0807271bf3abecf04f7579dfde 100644
--- a/sql/hive/src/test/resources/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8
+++ b/sql/hive/src/test/resources/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8
@@ -1,11 +1,11 @@
-col1                	string              	None                
-pcol1               	string              	None                
-pcol2               	string              	None                
+col1                	string              	                    
+pcol1               	string              	                    
+pcol2               	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-pcol1               	string              	None                
-pcol2               	string              	None                
+pcol1               	string              	                    
+pcol2               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter3_renamed, dbName:default, owner:marmbrus, createTime:1389728495, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter3_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, last_modified_by=marmbrus, last_modified_time=1389728509, transient_lastDdlTime=1389728509, numRows=6, totalSize=171, rawDataSize=6}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter3_renamed, dbName:default, owner:marmbrus, createTime:1413871632, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter3_renamed, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413871639, transient_lastDdlTime=1413871639}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43 b/sql/hive/src/test/resources/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43
index 1165de3a92fd038314655536e177613f08fa6105..658503a7518293dd5c14066f3cc7951da3d17080 100644
--- a/sql/hive/src/test/resources/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43
+++ b/sql/hive/src/test/resources/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43
@@ -1,11 +1,11 @@
-col1                	string              	None                
-pcol1               	string              	None                
-pcol2               	string              	None                
+col1                	string              	                    
+pcol1               	string              	                    
+pcol2               	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-pcol1               	string              	None                
-pcol2               	string              	None                
+pcol1               	string              	                    
+pcol2               	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[test_part:, test_part:], dbName:default, tableName:alter3_renamed, createTime:1389728508, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/alter3_renamed/pcol1=test_part%3A/pcol2=test_part%3A, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1389728508, numRows=6, totalSize=171, rawDataSize=6})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[test_part:, test_part:], dbName:default, tableName:alter3_renamed, createTime:1413871639, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:pcol1, type:string, comment:null), FieldSchema(name:pcol2, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter3_renamed/pcol1=test_part%3A/pcol2=test_part%3A, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413871639, COLUMN_STATS_ACCURATE=true, totalSize=171, numRows=6, rawDataSize=6})	
diff --git a/sql/hive/src/test/resources/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 b/sql/hive/src/test/resources/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2
index 8e609740f34a7f335bcbc34c865e04730bca5898..d1f5bd707fcb47cf84cd8d6d75b93f6e8ad86ece 100644
--- a/sql/hive/src/test/resources/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2
+++ b/sql/hive/src/test/resources/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2
@@ -3,4 +3,4 @@
 3	test_part:	test_part:
 4	test_part:	test_part:
 5	test_part:	test_part:
-6	test_part:	test_part:
\ No newline at end of file
+6	test_part:	test_part:
diff --git a/sql/hive/src/test/resources/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 b/sql/hive/src/test/resources/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7
index 8d2aebeb4d29d679939e59a2219de7b6fc624235..ab967d6b4295610c92e0e7c1fd4e7c6b966fed22 100644
--- a/sql/hive/src/test/resources/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7
+++ b/sql/hive/src/test/resources/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7
@@ -1,4 +1,4 @@
-key                 	int                 	None                
-value               	string              	None                
+key                 	int                 	                    
+value               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:set_bucketing_test, dbName:default, owner:marmbrus, createTime:1389735344, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5432448755546164619/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:10, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[key], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389735344}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:set_bucketing_test, dbName:default, owner:marmbrus, createTime:1413871664, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:10, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[key], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413871664}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 b/sql/hive/src/test/resources/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7
index cf67a009ff29144385594e35ca77423007c62420..9fafeb10086b2d688d84ebb327f06e3728fe769c 100644
--- a/sql/hive/src/test/resources/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7
+++ b/sql/hive/src/test/resources/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7
@@ -1,4 +1,4 @@
-key                 	int                 	None                
-value               	string              	None                
+key                 	int                 	                    
+value               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:set_bucketing_test, dbName:alter4_db, owner:marmbrus, createTime:1389735347, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5432448755546164619/alter4_db.db/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:10, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[key], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389735347}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:set_bucketing_test, dbName:alter4_db, owner:marmbrus, createTime:1413871664, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter4_db.db/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:10, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[key], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413871664}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 b/sql/hive/src/test/resources/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7
index a6375189a7a236435b6375fac838a0033e2ff050..576e37d08cd09f546ced88e35999ccac014d3fab 100644
--- a/sql/hive/src/test/resources/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7
+++ b/sql/hive/src/test/resources/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7
@@ -1,4 +1,4 @@
-key                 	int                 	None                
-value               	string              	None                
+key                 	int                 	                    
+value               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:set_bucketing_test, dbName:alter4_db, owner:marmbrus, createTime:1389735347, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5432448755546164619/alter4_db.db/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1389735348, transient_lastDdlTime=1389735348}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:set_bucketing_test, dbName:alter4_db, owner:marmbrus, createTime:1413871664, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter4_db.db/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871664, transient_lastDdlTime=1413871664, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 b/sql/hive/src/test/resources/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20
index 331d858ce9b12fa6720414196a9dd6e0b6a0faaa..4ad96d51599fb734101f6229f6c1a8a509bd6255 100644
--- a/sql/hive/src/test/resources/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20
+++ b/sql/hive/src/test/resources/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20
@@ -1 +1 @@
-default
\ No newline at end of file
+default
diff --git a/sql/hive/src/test/resources/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 b/sql/hive/src/test/resources/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7
index 6b79a37a85f0c4f97516060dca0493991b4f56af..0ed971efdf45009dd1ee3b441a541efbf4c00ba7 100644
--- a/sql/hive/src/test/resources/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7
+++ b/sql/hive/src/test/resources/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7
@@ -1,4 +1,4 @@
-key                 	int                 	None                
-value               	string              	None                
+key                 	int                 	                    
+value               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:set_bucketing_test, dbName:default, owner:marmbrus, createTime:1389735344, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5432448755546164619/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1389735345, transient_lastDdlTime=1389735345}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:set_bucketing_test, dbName:default, owner:marmbrus, createTime:1413871664, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/set_bucketing_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871664, transient_lastDdlTime=1413871664, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter4-5-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter4-5-9c36cac1372650b703400c60dd29042c
index ecafeaea5f61ab99d238c32150ce31e27145f46c..85c1918f46567d9bb13c443b5142296d0d2fbc14 100644
--- a/sql/hive/src/test/resources/golden/alter4-5-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/alter4-5-9c36cac1372650b703400c60dd29042c
@@ -1,2 +1,2 @@
 src
-srcpart
\ No newline at end of file
+srcpart
diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-3-cfef140167765d259320ed1e8aba718d b/sql/hive/src/test/resources/golden/alter5-1-cbad54fbd4a08fc8717708f93358ec3e
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_merge_2-3-cfef140167765d259320ed1e8aba718d
rename to sql/hive/src/test/resources/golden/alter5-1-cbad54fbd4a08fc8717708f93358ec3e
diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be b/sql/hive/src/test/resources/golden/alter5-15-cbad54fbd4a08fc8717708f93358ec3e
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be
rename to sql/hive/src/test/resources/golden/alter5-15-cbad54fbd4a08fc8717708f93358ec3e
diff --git a/sql/hive/src/test/resources/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf b/sql/hive/src/test/resources/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf
index e180d4c53ae73db43d149392ca144df7d3a6d447..b6aa69a915a6e9aa50406a678dc699f1a42f3229 100644
--- a/sql/hive/src/test/resources/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf
+++ b/sql/hive/src/test/resources/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf
@@ -1,9 +1,9 @@
-col1                	string              	None                
-dt                  	string              	None                
+col1                	string              	                    
+dt                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-dt                  	string              	None                
+dt                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[a], dbName:alter5_db, tableName:alter5, createTime:1390897177, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/alter5_db.db/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1390897177})	
+Detailed Partition Information	Partition(values:[a], dbName:alter5_db, tableName:alter5, createTime:1413871673, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter5_db.db/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413871673})	
diff --git a/sql/hive/src/test/resources/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf b/sql/hive/src/test/resources/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf
index 3d4ba2636ab66e283dd5dae61f13f86a89775b3e..1c321d52874729274369f3ae456c646c9258bbf5 100644
--- a/sql/hive/src/test/resources/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf
+++ b/sql/hive/src/test/resources/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf
@@ -1,9 +1,9 @@
-col1                	string              	None                
-dt                  	string              	None                
+col1                	string              	                    
+dt                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-dt                  	string              	None                
+dt                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[a], dbName:alter5_db, tableName:alter5, createTime:1390897177, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/alter5_db.db/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390897186, numRows=6, totalSize=12, rawDataSize=6})	
+Detailed Partition Information	Partition(values:[a], dbName:alter5_db, tableName:alter5, createTime:1413871673, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter5_db.db/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413871680, COLUMN_STATS_ACCURATE=true, totalSize=12, numRows=6, rawDataSize=6})	
diff --git a/sql/hive/src/test/resources/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf b/sql/hive/src/test/resources/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf
index 6669b628fc224567c864df04dccd6b7e2a792b85..110ee88b930d11cbe8da53f7845bb6c7c16a75a4 100644
--- a/sql/hive/src/test/resources/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf
+++ b/sql/hive/src/test/resources/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf
@@ -1,9 +1,9 @@
-col1                	string              	None                
-dt                  	string              	None                
+col1                	string              	                    
+dt                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-dt                  	string              	None                
+dt                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[a], dbName:default, tableName:alter5, createTime:1390897166, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1390897166})	
+Detailed Partition Information	Partition(values:[a], dbName:default, tableName:alter5, createTime:1413871665, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413871665})	
diff --git a/sql/hive/src/test/resources/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf b/sql/hive/src/test/resources/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf
index e866ae0fa76546ddc71837074415e70cc29a19b1..e7245656fec6c74ca9fa1db3440991c72b184ae5 100644
--- a/sql/hive/src/test/resources/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf
+++ b/sql/hive/src/test/resources/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf
@@ -1,9 +1,9 @@
-col1                	string              	None                
-dt                  	string              	None                
+col1                	string              	                    
+dt                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-dt                  	string              	None                
+dt                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[a], dbName:default, tableName:alter5, createTime:1390897166, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390897176, numRows=6, totalSize=12, rawDataSize=6})	
+Detailed Partition Information	Partition(values:[a], dbName:default, tableName:alter5, createTime:1413871665, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col1, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter5/parta, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413871672, COLUMN_STATS_ACCURATE=true, totalSize=12, numRows=6, rawDataSize=6})	
diff --git a/sql/hive/src/test/resources/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31 b/sql/hive/src/test/resources/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31
index 2860d5c6b4cfbf785a43aaa04f234b9192d722b5..1bfdd079010aa63654f0f9585c16c0cee0553a6c 100644
--- a/sql/hive/src/test/resources/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31
+++ b/sql/hive/src/test/resources/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31
@@ -1,5 +1,5 @@
-key                 	int                 	None                
+key                 	int                 	                    
 _bucketname         	string              	                    
 _offsets            	array<bigint>       	                    
 	 	 
-Detailed Table Information	Table(tableName:default__src_src_index_8__, dbName:default, owner:null, createTime:1389733869, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array<bigint>, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2023038695216118221/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389733869}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:default__src_src_index_8__, dbName:default, owner:marmbrus, createTime:1413871682, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array<bigint>, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413871682}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 b/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31
index 2860d5c6b4cfbf785a43aaa04f234b9192d722b5..1bfdd079010aa63654f0f9585c16c0cee0553a6c 100644
--- a/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31
+++ b/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31
@@ -1,5 +1,5 @@
-key                 	int                 	None                
+key                 	int                 	                    
 _bucketname         	string              	                    
 _offsets            	array<bigint>       	                    
 	 	 
-Detailed Table Information	Table(tableName:default__src_src_index_8__, dbName:default, owner:null, createTime:1389733869, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array<bigint>, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2023038695216118221/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389733869}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:default__src_src_index_8__, dbName:default, owner:marmbrus, createTime:1413871682, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array<bigint>, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413871682}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d b/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d
index ecafeaea5f61ab99d238c32150ce31e27145f46c..85c1918f46567d9bb13c443b5142296d0d2fbc14 100644
--- a/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d
+++ b/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d
@@ -1,2 +1,2 @@
 src
-srcpart
\ No newline at end of file
+srcpart
diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 b/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1
index c80ef36c96ad4ec24f14cd7feb2374da44ee1342..136628776692bcbe44d8fe20989bdfebbbb71f54 100644
--- a/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1
+++ b/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1
@@ -1 +1 @@
-754	-7678496319
\ No newline at end of file
+754	-7678496319
diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 b/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30
index 618c1d01b726d061d229816a1018b8bde8817f9b..d32a641c0050e2ea1fa82e1dea118645e77038e9 100644
--- a/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30
+++ b/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30
@@ -1,12 +1,12 @@
-key                 	int                 	from deserializer   
-value               	string              	from deserializer   
-ds                  	string              	None                
-ts                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+ts                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-ts                  	string              	None                
+ds                  	string              	                    
+ts                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[2012-01-03, 2012-01-03+14:46:31], dbName:default, tableName:src_rc_merge_test_part, createTime:1389728902, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/src_rc_merge_test_part/ds=2012-01-03/ts=2012-01-03+14%3A46%3A31, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1389728902})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[2012-01-03, 2012-01-03+14:46:31], dbName:default, tableName:src_rc_merge_test_part, createTime:1413930366, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6310217467672453297/src_rc_merge_test_part/ds=2012-01-03/ts=2012-01-03+14%3A46%3A31, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413930366})	
diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f b/sql/hive/src/test/resources/golden/alter_merge_2-3-bc9bb363f9a2026cfc70a31bb4551352
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f
rename to sql/hive/src/test/resources/golden/alter_merge_2-3-bc9bb363f9a2026cfc70a31bb4551352
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a b/sql/hive/src/test/resources/golden/alter_merge_2-4-d3bf7703ba00cf7c40f2a2dbb8ca7224
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a
rename to sql/hive/src/test/resources/golden/alter_merge_2-4-d3bf7703ba00cf7c40f2a2dbb8ca7224
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 b/sql/hive/src/test/resources/golden/alter_merge_2-5-6319bf26f3739260b1a77e2ea89ef147
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90
rename to sql/hive/src/test/resources/golden/alter_merge_2-5-6319bf26f3739260b1a77e2ea89ef147
diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 b/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02
index 3f10ffe7a4c473619c926cfb1e8d95e726e5a0ec..60d3b2f4a4cd5f1637eba020358bfe5ecb5edcf2 100644
--- a/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02
+++ b/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02
@@ -1 +1 @@
-15
\ No newline at end of file
+15
diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 b/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1
index c80ef36c96ad4ec24f14cd7feb2374da44ee1342..136628776692bcbe44d8fe20989bdfebbbb71f54 100644
--- a/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1
+++ b/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1
@@ -1 +1 @@
-754	-7678496319
\ No newline at end of file
+754	-7678496319
diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 b/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02
index 3f10ffe7a4c473619c926cfb1e8d95e726e5a0ec..60d3b2f4a4cd5f1637eba020358bfe5ecb5edcf2 100644
--- a/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02
+++ b/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02
@@ -1 +1 @@
-15
\ No newline at end of file
+15
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 b/sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256
deleted file mode 100644
index 410b14d2ce6f958c13adcb30807e8673074c49d9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256
+++ /dev/null
@@ -1 +0,0 @@
-25
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 b/sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58
deleted file mode 100644
index 410b14d2ce6f958c13adcb30807e8673074c49d9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58
+++ /dev/null
@@ -1 +0,0 @@
-25
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e b/sql/hive/src/test/resources/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e
deleted file mode 100644
index b28ab5ccf8a1bc6aa4a761dcfbaabe543c335f8f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e
+++ /dev/null
@@ -1,75 +0,0 @@
-238	val_238	10	3.0
-NULL		10	3.0
-311	val_311	10	3.0
-NULL	val_27	10	3.0
-NULL	val_165	10	3.0
-NULL	val_409	10	3.0
-255	val_255	10	3.0
-278	val_278	10	3.0
-98	val_98	10	3.0
-NULL	val_484	10	3.0
-NULL	val_265	10	3.0
-NULL	val_193	10	3.0
-401	val_401	10	3.0
-150	val_150	10	3.0
-273	val_273	10	3.0
-224		10	3.0
-369		10	3.0
-66	val_66	10	3.0
-128		10	3.0
-213	val_213	10	3.0
-146	val_146	10	3.0
-406	val_406	10	3.0
-NULL		10	3.0
-NULL		10	3.0
-NULL		10	3.0
-238	val_238	100x	3.0
-NULL		100x	3.0
-311	val_311	100x	3.0
-NULL	val_27	100x	3.0
-NULL	val_165	100x	3.0
-NULL	val_409	100x	3.0
-255	val_255	100x	3.0
-278	val_278	100x	3.0
-98	val_98	100x	3.0
-NULL	val_484	100x	3.0
-NULL	val_265	100x	3.0
-NULL	val_193	100x	3.0
-401	val_401	100x	3.0
-150	val_150	100x	3.0
-273	val_273	100x	3.0
-224		100x	3.0
-369		100x	3.0
-66	val_66	100x	3.0
-128		100x	3.0
-213	val_213	100x	3.0
-146	val_146	100x	3.0
-406	val_406	100x	3.0
-NULL		100x	3.0
-NULL		100x	3.0
-NULL		100x	3.0
-238	val_238	100x	6:30pm
-NULL		100x	6:30pm
-311	val_311	100x	6:30pm
-NULL	val_27	100x	6:30pm
-NULL	val_165	100x	6:30pm
-NULL	val_409	100x	6:30pm
-255	val_255	100x	6:30pm
-278	val_278	100x	6:30pm
-98	val_98	100x	6:30pm
-NULL	val_484	100x	6:30pm
-NULL	val_265	100x	6:30pm
-NULL	val_193	100x	6:30pm
-401	val_401	100x	6:30pm
-150	val_150	100x	6:30pm
-273	val_273	100x	6:30pm
-224		100x	6:30pm
-369		100x	6:30pm
-66	val_66	100x	6:30pm
-128		100x	6:30pm
-213	val_213	100x	6:30pm
-146	val_146	100x	6:30pm
-406	val_406	100x	6:30pm
-NULL		100x	6:30pm
-NULL		100x	6:30pm
-NULL		100x	6:30pm
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c b/sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c
deleted file mode 100644
index c5b431b6cba29540b4b284840ff229bce0460886..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c
+++ /dev/null
@@ -1 +0,0 @@
-50
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 b/sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3
deleted file mode 100644
index a76c74dcec6ab8ba53c5d5b4de054eb43f68fe67..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3
+++ /dev/null
@@ -1 +0,0 @@
-75
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a b/sql/hive/src/test/resources/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a
deleted file mode 100644
index 316ca7f65ba20ce52e785929123b3e0f9a918c4d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a
+++ /dev/null
@@ -1,10 +0,0 @@
-key                 	string              	None                
-value               	string              	None                
-dt                  	string              	None                
-ts                  	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-dt                  	string              	None                
-ts                  	string              	None                
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a b/sql/hive/src/test/resources/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a
deleted file mode 100644
index a7382fabfcb4947e76e03715ef2d9c4d78342e71..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a
+++ /dev/null
@@ -1,10 +0,0 @@
-key                 	string              	None                
-value               	string              	None                
-dt                  	string              	None                
-ts                  	double              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-dt                  	string              	None                
-ts                  	double              	None                
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 b/sql/hive/src/test/resources/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93
deleted file mode 100644
index a7382fabfcb4947e76e03715ef2d9c4d78342e71..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93
+++ /dev/null
@@ -1,10 +0,0 @@
-key                 	string              	None                
-value               	string              	None                
-dt                  	string              	None                
-ts                  	double              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-dt                  	string              	None                
-ts                  	double              	None                
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a b/sql/hive/src/test/resources/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a
deleted file mode 100644
index a7382fabfcb4947e76e03715ef2d9c4d78342e71..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a
+++ /dev/null
@@ -1,10 +0,0 @@
-key                 	string              	None                
-value               	string              	None                
-dt                  	string              	None                
-ts                  	double              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-dt                  	string              	None                
-ts                  	double              	None                
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 b/sql/hive/src/test/resources/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6
deleted file mode 100644
index a7382fabfcb4947e76e03715ef2d9c4d78342e71..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6
+++ /dev/null
@@ -1,10 +0,0 @@
-key                 	string              	None                
-value               	string              	None                
-dt                  	string              	None                
-ts                  	double              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-dt                  	string              	None                
-ts                  	double              	None                
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 b/sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5
deleted file mode 100644
index 410b14d2ce6f958c13adcb30807e8673074c49d9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5
+++ /dev/null
@@ -1 +0,0 @@
-25
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd b/sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd
deleted file mode 100644
index 410b14d2ce6f958c13adcb30807e8673074c49d9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd
+++ /dev/null
@@ -1 +0,0 @@
-25
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 b/sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8
index db182e444d31d01628bde22e306b3df136d960ef..11487abed2b60df7a1b634d1e1053f3ac45078a8 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8
@@ -1,4 +1,4 @@
-key                 	int                 	None                
-value               	string              	None                
+key                 	int                 	                    
+value               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871688, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413871688}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321
index 81e23f0bc19512705c2e6c28cab46245d5665c94..979969dcbfd3f8006fbbcb182b8e6f20225c0a3e 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321
@@ -1,10 +1,10 @@
-key                 	int                 	from deserializer   
-value               	string              	from deserializer   
-ds                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
+ds                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1388805891, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1413871689, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1})	
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321
index 6dc1f3ca2c1875a0e4134ea75ffc8be068048f3a..7e14edcdead2e3ba8c0838aa532e8f6c3ce37cf6 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321
@@ -1,10 +1,10 @@
-key                 	int                 	from deserializer   
-value               	string              	from deserializer   
-ds                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
+ds                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1388805891, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1413871689, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/test/test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1})	
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8
index 4b754043d63ab21f306b35ad9d1a899c1420e7ab..77a764a814eb939b9d4d8812dc0a9d0b2fae75a9 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8
@@ -1,10 +1,10 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
+ds                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871689, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1413871689}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8
index f44c28ee367608ee5cc5c697954a98c374c4a320..c8606b1acad0cb3b2962ff06907f416cf1468673 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8
@@ -1,10 +1,10 @@
-key                 	int                 	from deserializer   
-value               	string              	from deserializer   
-ds                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
+ds                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388805892, transient_lastDdlTime=1388805892}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871689, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8
index e739ad4992ec9d47dfea9b7a271b04a1ff52a744..59922d3b7a0860167681a791f0665e4f6c2e839e 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8
@@ -1,10 +1,10 @@
-key                 	int                 	from deserializer   
-value               	string              	from deserializer   
-ds                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
+ds                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388805892, transient_lastDdlTime=1388805892}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871689, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8
index 092e5ed6e8a46e80ebda459f5d9028d13c3e4eb3..45ef7555394796e32b0f46b73c435988736d5e04 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8
@@ -1,4 +1,4 @@
-key                 	int                 	from deserializer   
-value               	string              	from deserializer   
+key                 	int                 	                    
+value               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871688, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871688, transient_lastDdlTime=1413871688, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8
index 197e67d09bf49e331588cea647287d23750ce6c5..d6804307f3dc72787f762aa3b9ef42b8d5e06ada 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8
@@ -1,4 +1,4 @@
-key                 	int                 	from deserializer   
-value               	string              	from deserializer   
+key                 	int                 	                    
+value               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871688, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871688, transient_lastDdlTime=1413871688, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321
index 29b5b693b8589e908d786660d15cd5b3fabfab6e..77ba51afd246838a67e5cf5dcf24f7f8fbb3b55e 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321
+++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321
@@ -1,10 +1,10 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
+ds                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1388805891, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805891})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1413871689, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413871689})	
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-e11f1476d2f30ab7a83f95013809f9e6
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9
rename to sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-e11f1476d2f30ab7a83f95013809f9e6
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-29b4b67965f24c40bcc426043efb892d
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6
rename to sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-29b4b67965f24c40bcc426043efb892d
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-2100fc7853625d8c7dad5c0ee70d4690
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e
rename to sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-2100fc7853625d8c7dad5c0ee70d4690
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-c3fa911536c75e87febf2874e7889879
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880
rename to sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-c3fa911536c75e87febf2874e7889879
diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17
index 02d72f4292749f3176b772246d7f3ff19fc55b5f..c1b66aba46e84bfe7723ae2554d04b29fb6e96df 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17
+++ b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17
@@ -9,4 +9,4 @@
 3	13	1996	12
 7	17	1996	12
 8	18	1996	12
-8	28	1996	12
\ No newline at end of file
+8	28	1996	12
diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d
index bca47334cedaa9f354734d87729927510bc0f800..12ec4459dc1671a9788a73545f97ab17cd936a42 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d
+++ b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d
@@ -3,4 +3,4 @@
 3	13	1995	09
 7	17	1995	09
 8	18	1995	09
-8	28	1995	09
\ No newline at end of file
+8	28	1995	09
diff --git a/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 b/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196
+++ b/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 b/sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7
index e881b2a4b74ebde503d7b1a3366d3ee70a04069d..605b17f1d8cebe9a2cca7cdc02fc99c4389f9636 100644
--- a/sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7
+++ b/sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7
@@ -3,4 +3,4 @@
 3	new_part1:	new_part2:
 4	new_part1:	new_part2:
 5	new_part1:	new_part2:
-6	new_part1:	new_part2:
\ No newline at end of file
+6	new_part1:	new_part2:
diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c
index ecafeaea5f61ab99d238c32150ce31e27145f46c..85c1918f46567d9bb13c443b5142296d0d2fbc14 100644
--- a/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c
@@ -1,2 +1,2 @@
 src
-srcpart
\ No newline at end of file
+srcpart
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f b/sql/hive/src/test/resources/golden/alter_rename_partition-19-d862c00e5a37ae841a6c8ec6c5d68e8c
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f
rename to sql/hive/src/test/resources/golden/alter_rename_partition-19-d862c00e5a37ae841a6c8ec6c5d68e8c
diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c
index ecafeaea5f61ab99d238c32150ce31e27145f46c..85c1918f46567d9bb13c443b5142296d0d2fbc14 100644
--- a/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c
@@ -1,2 +1,2 @@
 src
-srcpart
\ No newline at end of file
+srcpart
diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 b/sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2
index 684f1da0fd0a6ee9dfd6b2534746ea2416dc8435..6105de433830ce7e8764ab8c466f03ab72d80947 100644
--- a/sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2
+++ b/sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2
@@ -3,4 +3,4 @@
 3	old_part1:	old_part2:
 4	old_part1:	old_part2:
 5	old_part1:	old_part2:
-6	old_part1:	old_part2:
\ No newline at end of file
+6	old_part1:	old_part2:
diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 b/sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3
index fc31ec62a1280b5f3d9bb30b592bb450a6dcd4dd..fb196b30f44780d529c332dfb2a8e0ca41385c1b 100644
--- a/sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3
+++ b/sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3
@@ -1 +1 @@
-pcol1=new_part1%3A/pcol2=new_part2%3A
\ No newline at end of file
+pcol1=new_part1%3A/pcol2=new_part2%3A
diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b b/sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b
index e881b2a4b74ebde503d7b1a3366d3ee70a04069d..605b17f1d8cebe9a2cca7cdc02fc99c4389f9636 100644
--- a/sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b
+++ b/sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b
@@ -3,4 +3,4 @@
 3	new_part1:	new_part2:
 4	new_part1:	new_part2:
 5	new_part1:	new_part2:
-6	new_part1:	new_part2:
\ No newline at end of file
+6	new_part1:	new_part2:
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 b/sql/hive/src/test/resources/golden/alter_rename_partition-4-833254c8f7c5b1248e1a249560652627
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30
rename to sql/hive/src/test/resources/golden/alter_rename_partition-4-833254c8f7c5b1248e1a249560652627
diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 b/sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7
index 684f1da0fd0a6ee9dfd6b2534746ea2416dc8435..6105de433830ce7e8764ab8c466f03ab72d80947 100644
--- a/sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7
+++ b/sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7
@@ -3,4 +3,4 @@
 3	old_part1:	old_part2:
 4	old_part1:	old_part2:
 5	old_part1:	old_part2:
-6	old_part1:	old_part2:
\ No newline at end of file
+6	old_part1:	old_part2:
diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 b/sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3
index fc31ec62a1280b5f3d9bb30b592bb450a6dcd4dd..fb196b30f44780d529c332dfb2a8e0ca41385c1b 100644
--- a/sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3
+++ b/sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3
@@ -1 +1 @@
-pcol1=new_part1%3A/pcol2=new_part2%3A
\ No newline at end of file
+pcol1=new_part1%3A/pcol2=new_part2%3A
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879
index ccd6518a50f7c680816f0dbdab80fa2d73598ab0..5a608a9064b8407d1fc1587419b858f5d95b5e7d 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879
@@ -1,5 +1,5 @@
-id                  	int                 	None                
-query               	string              	None                
-name                	string              	None                
+id                  	int                 	                    
+query               	string              	                    
+name                	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1388805893, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1388805893}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1413871711, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413871711}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8
index d135e450e6e8e9e7c4cb84feb1105532ba7b7aee..1d6d48bbe1d1ccd45daebdcfd2b897eaf60bccd9 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8
@@ -1,11 +1,11 @@
-id                  	int                 	None                
-query               	string              	None                
-name                	string              	None                
-dt                  	string              	None                
+id                  	int                 	                    
+query               	string              	                    
+name                	string              	                    
+dt                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-dt                  	string              	None                
+dt                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805893})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413871711})	
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8
index d135e450e6e8e9e7c4cb84feb1105532ba7b7aee..1d6d48bbe1d1ccd45daebdcfd2b897eaf60bccd9 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8
@@ -1,11 +1,11 @@
-id                  	int                 	None                
-query               	string              	None                
-name                	string              	None                
-dt                  	string              	None                
+id                  	int                 	                    
+query               	string              	                    
+name                	string              	                    
+dt                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-dt                  	string              	None                
+dt                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805893})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413871711})	
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8
index c3fac88f91a36ec9edbfa2eeddb7d968ecace999..f33269bd266be8917c63aab40da0b618748cbc51 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8
@@ -1,11 +1,11 @@
-id                  	int                 	None                
-query               	string              	None                
-name                	string              	None                
-dt                  	string              	None                
+id                  	int                 	                    
+query               	string              	                    
+name                	string              	                    
+dt                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-dt                  	string              	None                
+dt                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871712, transient_lastDdlTime=1413871712, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1})	
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8
index 6c8f91de7cdedb5a11f7954ee02ace65c0f2c9be..41e8d696d33ea44edcdd29f9c83c8037346cbd0f 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8
@@ -1,11 +1,11 @@
-id                  	int                 	None                
-query               	string              	None                
-name                	string              	None                
-dt                  	string              	None                
+id                  	int                 	                    
+query               	string              	                    
+name                	string              	                    
+dt                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-dt                  	string              	None                
+dt                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871712, transient_lastDdlTime=1413871712, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1})	
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879
index 37a5b2cc47bad187f7ff2687a8130e0f55bcc50c..7caf0af50b071b40cfd908250b0048b2618796d7 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879
@@ -1,5 +1,5 @@
-id                  	int                 	from deserializer   
-query               	string              	from deserializer   
-name                	string              	from deserializer   
+id                  	int                 	                    
+query               	string              	                    
+name                	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1388805893, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:from deserializer), FieldSchema(name:query, type:string, comment:from deserializer), FieldSchema(name:name, type:string, comment:from deserializer)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1413871711, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871711, transient_lastDdlTime=1413871711, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879
index 0348dd15fd4f5a30024b75f7335f54001a8af7ec..a55479e69bbeaea80885e772c6232e79cd06c03e 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879
@@ -1,5 +1,5 @@
-id                  	int                 	from deserializer   
-query               	string              	from deserializer   
-name                	string              	from deserializer   
+id                  	int                 	                    
+query               	string              	                    
+name                	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1388805893, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:from deserializer), FieldSchema(name:query, type:string, comment:from deserializer), FieldSchema(name:name, type:string, comment:from deserializer)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1413871711, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871711, transient_lastDdlTime=1413871711, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8
index d135e450e6e8e9e7c4cb84feb1105532ba7b7aee..1d6d48bbe1d1ccd45daebdcfd2b897eaf60bccd9 100644
--- a/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8
+++ b/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8
@@ -1,11 +1,11 @@
-id                  	int                 	None                
-query               	string              	None                
-name                	string              	None                
-dt                  	string              	None                
+id                  	int                 	                    
+query               	string              	                    
+name                	string              	                    
+dt                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-dt                  	string              	None                
+dt                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805893})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413871711})	
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 b/sql/hive/src/test/resources/golden/alter_varchar2-2-3a20c238eab602ad3d593b1eb3fa6dbb
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6
rename to sql/hive/src/test/resources/golden/alter_varchar2-2-3a20c238eab602ad3d593b1eb3fa6dbb
diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 b/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797
index 40818a7de46d05013d1359668e62bfc34dce6dd7..600b3777168945d54fd963985f0a07aac747df93 100644
--- a/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797
+++ b/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797
@@ -1 +1 @@
-val_238	7
\ No newline at end of file
+val_238	7
diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c b/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c
index 827220bd4996fcafb779cbebd836a9ba814196c3..ad69f390bc8db1561ddb11430766f25cf59adbd8 100644
--- a/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c
+++ b/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c
@@ -1 +1 @@
-1	val_238	7
\ No newline at end of file
+1	val_238	7
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d b/sql/hive/src/test/resources/golden/alter_varchar2-6-3250407f20f3766c18f44b8bfae1829d
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d
rename to sql/hive/src/test/resources/golden/alter_varchar2-6-3250407f20f3766c18f44b8bfae1829d
diff --git a/sql/hive/src/test/resources/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 b/sql/hive/src/test/resources/golden/alter_varchar2-7-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54
rename to sql/hive/src/test/resources/golden/alter_varchar2-7-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c b/sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c
deleted file mode 100644
index 827220bd4996fcafb779cbebd836a9ba814196c3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c
+++ /dev/null
@@ -1 +0,0 @@
-1	val_238	7
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c b/sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c
deleted file mode 100644
index 8a8234a35f6bb05d43d5adce9e16aca7211c1efd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c
+++ /dev/null
@@ -1 +0,0 @@
-2	238	3
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-8-84e700f9dc6033c1f237fcdb95e31a0c b/sql/hive/src/test/resources/golden/alter_varchar2-8-84e700f9dc6033c1f237fcdb95e31a0c
new file mode 100644
index 0000000000000000000000000000000000000000..ad69f390bc8db1561ddb11430766f25cf59adbd8
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/alter_varchar2-8-84e700f9dc6033c1f237fcdb95e31a0c
@@ -0,0 +1 @@
+1	val_238	7
diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-9-4c12c4c53d99338796be34e603dc612c b/sql/hive/src/test/resources/golden/alter_varchar2-9-4c12c4c53d99338796be34e603dc612c
new file mode 100644
index 0000000000000000000000000000000000000000..1f8ddaec9003a2d1b00c2601667eaf03622c7492
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/alter_varchar2-9-4c12c4c53d99338796be34e603dc612c
@@ -0,0 +1 @@
+2	238	3
diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b
index a99747531cef47309accaa923670bf757afd0ca1..ecae907d9227966dd95a204d59f58f3c027eafc0 100644
--- a/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b
+++ b/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b
@@ -1,20 +1,20 @@
 # col_name            	data_type           	comment             
 	 	 
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Fri Feb 07 14:47:52 PST 2014	 
+CreateTime:         	Mon Oct 20 23:10:25 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1391813272          
+	transient_lastDdlTime	1413871825          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	null                	 
diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b
index 499c73127d890c3158aecaf826bedffed119d46e..10e3eca6c67912c59967587a975caa7ee6571bec 100644
--- a/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b
+++ b/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b
@@ -1,17 +1,17 @@
 # col_name            	data_type           	comment             
 	 	 
-value               	string              	None                
+value               	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Fri Feb 07 14:47:52 PST 2014	 
+CreateTime:         	Mon Oct 20 23:10:25 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1391813272          
+	transient_lastDdlTime	1413871825          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	null                	 
diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b
index a5fba77abdf07b50edd074dc88fd6c9fe3cbeddd..c3661a1f79b5776f71b0994353fc5db0b2046e31 100644
--- a/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b
+++ b/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b
@@ -1,18 +1,18 @@
 # col_name            	data_type           	comment             
 	 	 
-key                 	int                 	None                
-value               	string              	None                
+key                 	int                 	                    
+value               	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Fri Feb 07 14:47:52 PST 2014	 
+CreateTime:         	Mon Oct 20 23:10:25 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
 Table Type:         	VIRTUAL_VIEW        	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1391813272          
+	transient_lastDdlTime	1413871825          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	null                	 
diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb b/sql/hive/src/test/resources/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb
deleted file mode 100644
index ee76e02af3abafb19b1ad23e85926120361d21a3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb
+++ /dev/null
@@ -1,5 +0,0 @@
-foo                 	int                 	None                
-bar                 	string              	None                
-ds                  	string              	None                
-	 	 
-Detailed Table Information	Table(tableName:view1, dbName:default, owner:tnachen, createTime:1392426511, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:foo, type:int, comment:null), FieldSchema(name:bar, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:null, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:null, parameters:{}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426511}, viewOriginalText:SELECT * FROM invites, viewExpandedText:SELECT `invites`.`foo`, `invites`.`bar`, `invites`.`ds` FROM `default`.`invites`, tableType:VIRTUAL_VIEW)	
diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 b/sql/hive/src/test/resources/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3
deleted file mode 100644
index 8603577477bfc8f8aa71e71e2ba98277b6c08616..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3
+++ /dev/null
@@ -1,5 +0,0 @@
-foo                 	int                 	None                
-bar                 	string              	None                
-ds                  	string              	None                
-	 	 
-Detailed Table Information	Table(tableName:view2, dbName:default, owner:tnachen, createTime:1392426511, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:foo, type:int, comment:null), FieldSchema(name:bar, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:null, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:null, parameters:{}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=tnachen, last_modified_time=1392426511, transient_lastDdlTime=1392426511}, viewOriginalText:SELECT * FROM invites, viewExpandedText:SELECT `invites`.`foo`, `invites`.`bar`, `invites`.`ds` FROM `default`.`invites`, tableType:VIRTUAL_VIEW)	
diff --git a/sql/hive/src/test/resources/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/ambiguous_col-0-e91e3e5a22029b9b979ccbbef97add66
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c
rename to sql/hive/src/test/resources/golden/ambiguous_col-0-e91e3e5a22029b9b979ccbbef97add66
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d b/sql/hive/src/test/resources/golden/ambiguous_col-1-b4fe82679efdf6a15e9ecff53baf8d8d
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d
rename to sql/hive/src/test/resources/golden/ambiguous_col-1-b4fe82679efdf6a15e9ecff53baf8d8d
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 b/sql/hive/src/test/resources/golden/ambiguous_col-2-dadfa3854356dead14b93c5a71a5d8ab
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0
rename to sql/hive/src/test/resources/golden/ambiguous_col-2-dadfa3854356dead14b93c5a71a5d8ab
diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 b/sql/hive/src/test/resources/golden/ambiguous_col-3-70509ccd2765d90b98666b6dff8afe1b
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42
rename to sql/hive/src/test/resources/golden/ambiguous_col-3-70509ccd2765d90b98666b6dff8afe1b
diff --git a/sql/hive/src/test/resources/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/annotate_stats_join-0-2a0c41508e1e70eaedf9de99751c8fa9
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0
rename to sql/hive/src/test/resources/golden/annotate_stats_join-0-2a0c41508e1e70eaedf9de99751c8fa9
diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c b/sql/hive/src/test/resources/golden/annotate_stats_join-1-84e7846d50fc15e836c83911ce039871
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c
rename to sql/hive/src/test/resources/golden/annotate_stats_join-1-84e7846d50fc15e836c83911ce039871
diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 b/sql/hive/src/test/resources/golden/annotate_stats_join-10-a59dc1c01d48c82d46a5beab759f618d
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56
rename to sql/hive/src/test/resources/golden/annotate_stats_join-10-a59dc1c01d48c82d46a5beab759f618d
diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 b/sql/hive/src/test/resources/golden/annotate_stats_join-11-3e0ade2476221f6396381f55a3b82e60
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4
rename to sql/hive/src/test/resources/golden/annotate_stats_join-11-3e0ade2476221f6396381f55a3b82e60
diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 b/sql/hive/src/test/resources/golden/annotate_stats_join-12-cef44682d339a67ba765f854da21f976
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6
rename to sql/hive/src/test/resources/golden/annotate_stats_join-12-cef44682d339a67ba765f854da21f976
diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c b/sql/hive/src/test/resources/golden/annotate_stats_join-13-6292c27f0c4eaeab2a5e6c317e3afa2e
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c
rename to sql/hive/src/test/resources/golden/annotate_stats_join-13-6292c27f0c4eaeab2a5e6c317e3afa2e
diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f b/sql/hive/src/test/resources/golden/annotate_stats_join-14-4f3042b9feebd00c540ddac40e7254d1
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f
rename to sql/hive/src/test/resources/golden/annotate_stats_join-14-4f3042b9feebd00c540ddac40e7254d1
diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 b/sql/hive/src/test/resources/golden/annotate_stats_join-15-a2f3b8a636e46e5df514df46c452855f
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19
rename to sql/hive/src/test/resources/golden/annotate_stats_join-15-a2f3b8a636e46e5df514df46c452855f
diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 b/sql/hive/src/test/resources/golden/annotate_stats_join-16-a75699a21ea36d962a1ba45bd5a12f26
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656
rename to sql/hive/src/test/resources/golden/annotate_stats_join-16-a75699a21ea36d962a1ba45bd5a12f26
diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 b/sql/hive/src/test/resources/golden/annotate_stats_join-17-64380f94a431e3a1698aa2edd3d0a6b2
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6
rename to sql/hive/src/test/resources/golden/annotate_stats_join-17-64380f94a431e3a1698aa2edd3d0a6b2
diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d b/sql/hive/src/test/resources/golden/annotate_stats_join-18-222d1fcce7420950436e00d3a1bba957
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d
rename to sql/hive/src/test/resources/golden/annotate_stats_join-18-222d1fcce7420950436e00d3a1bba957
diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a b/sql/hive/src/test/resources/golden/annotate_stats_join-19-dea5f8f5c703583c7a3bdcb62cd3d589
similarity index 100%
rename from sql/hive/src/test/resources/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a
rename to sql/hive/src/test/resources/golden/annotate_stats_join-19-dea5f8f5c703583c7a3bdcb62cd3d589
diff --git a/sql/hive/src/test/resources/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d b/sql/hive/src/test/resources/golden/annotate_stats_join-2-40548ec2313af8dbdcbb8ad0477d8600
similarity index 100%
rename from sql/hive/src/test/resources/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d
rename to sql/hive/src/test/resources/golden/annotate_stats_join-2-40548ec2313af8dbdcbb8ad0477d8600
diff --git a/sql/hive/src/test/resources/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab b/sql/hive/src/test/resources/golden/annotate_stats_join-20-2d5e186b97166e3a625a169d0d73f9c8
similarity index 100%
rename from sql/hive/src/test/resources/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab
rename to sql/hive/src/test/resources/golden/annotate_stats_join-20-2d5e186b97166e3a625a169d0d73f9c8
diff --git a/sql/hive/src/test/resources/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b b/sql/hive/src/test/resources/golden/annotate_stats_join-21-ed73d4b01424287148347ccf721b37e2
similarity index 100%
rename from sql/hive/src/test/resources/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b
rename to sql/hive/src/test/resources/golden/annotate_stats_join-21-ed73d4b01424287148347ccf721b37e2
diff --git a/sql/hive/src/test/resources/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 b/sql/hive/src/test/resources/golden/annotate_stats_join-22-2cf93da6bb0efdafeaa989c9df236701
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1
rename to sql/hive/src/test/resources/golden/annotate_stats_join-22-2cf93da6bb0efdafeaa989c9df236701
diff --git a/sql/hive/src/test/resources/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/annotate_stats_join-23-fa90806f6137300311df032e28df3d4c
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-12-f07653bca86e1ecb614ffd0296790d05
rename to sql/hive/src/test/resources/golden/annotate_stats_join-23-fa90806f6137300311df032e28df3d4c
diff --git a/sql/hive/src/test/resources/golden/archive-16-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/annotate_stats_join-24-4a6976344eeae35e059285ed78f9feb3
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-16-892147913578bcf60620b7dd73893dd0
rename to sql/hive/src/test/resources/golden/annotate_stats_join-24-4a6976344eeae35e059285ed78f9feb3
diff --git a/sql/hive/src/test/resources/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a b/sql/hive/src/test/resources/golden/annotate_stats_join-3-26f82fb6734d5bc6f7159b06c0949178
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a
rename to sql/hive/src/test/resources/golden/annotate_stats_join-3-26f82fb6734d5bc6f7159b06c0949178
diff --git a/sql/hive/src/test/resources/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f b/sql/hive/src/test/resources/golden/annotate_stats_join-4-a598c93d86a646cfa6ea86da54bce9b8
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f
rename to sql/hive/src/test/resources/golden/annotate_stats_join-4-a598c93d86a646cfa6ea86da54bce9b8
diff --git a/sql/hive/src/test/resources/golden/archive-2-713efc113418b01f76ffd589840193c8 b/sql/hive/src/test/resources/golden/annotate_stats_join-5-d12ba848d20d1def95379c490068f050
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-2-713efc113418b01f76ffd589840193c8
rename to sql/hive/src/test/resources/golden/annotate_stats_join-5-d12ba848d20d1def95379c490068f050
diff --git a/sql/hive/src/test/resources/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/annotate_stats_join-6-96a4806e61c5d882affac57e115f285f
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-21-f07653bca86e1ecb614ffd0296790d05
rename to sql/hive/src/test/resources/golden/annotate_stats_join-6-96a4806e61c5d882affac57e115f285f
diff --git a/sql/hive/src/test/resources/golden/archive-23-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/annotate_stats_join-7-38be41d6328f57350a78fb2097d1ebd2
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-23-892147913578bcf60620b7dd73893dd0
rename to sql/hive/src/test/resources/golden/annotate_stats_join-7-38be41d6328f57350a78fb2097d1ebd2
diff --git a/sql/hive/src/test/resources/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f b/sql/hive/src/test/resources/golden/annotate_stats_join-8-eb11e867a36e2114ef0f0f001e01984c
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f
rename to sql/hive/src/test/resources/golden/annotate_stats_join-8-eb11e867a36e2114ef0f0f001e01984c
diff --git a/sql/hive/src/test/resources/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a b/sql/hive/src/test/resources/golden/annotate_stats_join-9-e0b7cb2226234fbea8ae11ea7d9d84bd
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a
rename to sql/hive/src/test/resources/golden/annotate_stats_join-9-e0b7cb2226234fbea8ae11ea7d9d84bd
diff --git a/sql/hive/src/test/resources/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/annotate_stats_limit-0-2a0c41508e1e70eaedf9de99751c8fa9
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-15-7cccbdffc32975f8935eeba14a28147
rename to sql/hive/src/test/resources/golden/annotate_stats_limit-0-2a0c41508e1e70eaedf9de99751c8fa9
diff --git a/sql/hive/src/test/resources/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 b/sql/hive/src/test/resources/golden/annotate_stats_limit-1-26f82fb6734d5bc6f7159b06c0949178
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5
rename to sql/hive/src/test/resources/golden/annotate_stats_limit-1-26f82fb6734d5bc6f7159b06c0949178
diff --git a/sql/hive/src/test/resources/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 b/sql/hive/src/test/resources/golden/annotate_stats_limit-10-a89c94fd0de0cfc96725fea890987cc0
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19
rename to sql/hive/src/test/resources/golden/annotate_stats_limit-10-a89c94fd0de0cfc96725fea890987cc0
diff --git a/sql/hive/src/test/resources/golden/archive-3-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_limit-2-eb11e867a36e2114ef0f0f001e01984c
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-3-27ad2962fed131f51ba802596ba37278
rename to sql/hive/src/test/resources/golden/annotate_stats_limit-2-eb11e867a36e2114ef0f0f001e01984c
diff --git a/sql/hive/src/test/resources/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/annotate_stats_limit-3-e0b7cb2226234fbea8ae11ea7d9d84bd
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-31-cd46bc635e3010cf1b990a652a584a09
rename to sql/hive/src/test/resources/golden/annotate_stats_limit-3-e0b7cb2226234fbea8ae11ea7d9d84bd
diff --git a/sql/hive/src/test/resources/golden/archive-32-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_limit-4-4252006172a476fbc591eebee49bffa3
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-32-27ad2962fed131f51ba802596ba37278
rename to sql/hive/src/test/resources/golden/annotate_stats_limit-4-4252006172a476fbc591eebee49bffa3
diff --git a/sql/hive/src/test/resources/golden/archive-4-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/annotate_stats_limit-5-a2f3b8a636e46e5df514df46c452855f
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-4-3e95421993ab28d18245ec2340f580a3
rename to sql/hive/src/test/resources/golden/annotate_stats_limit-5-a2f3b8a636e46e5df514df46c452855f
diff --git a/sql/hive/src/test/resources/golden/archive-5-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/annotate_stats_limit-6-f1fd6c403a9787947877f48c772afe96
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-5-c0c18ac884677231a41eea8d980d0451
rename to sql/hive/src/test/resources/golden/annotate_stats_limit-6-f1fd6c403a9787947877f48c772afe96
diff --git a/sql/hive/src/test/resources/golden/archive-6-528ab9750a558af7f1a43b3108e793dd b/sql/hive/src/test/resources/golden/annotate_stats_limit-7-69b422026fce4e2d2cde9a478d6aaa40
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-6-528ab9750a558af7f1a43b3108e793dd
rename to sql/hive/src/test/resources/golden/annotate_stats_limit-7-69b422026fce4e2d2cde9a478d6aaa40
diff --git a/sql/hive/src/test/resources/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca b/sql/hive/src/test/resources/golden/annotate_stats_limit-8-72f5e55c1e244084eea255c32a6f58eb
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca
rename to sql/hive/src/test/resources/golden/annotate_stats_limit-8-72f5e55c1e244084eea255c32a6f58eb
diff --git a/sql/hive/src/test/resources/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c b/sql/hive/src/test/resources/golden/annotate_stats_limit-9-9da67c62d5e3145d450ad7953b33361f
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c
rename to sql/hive/src/test/resources/golden/annotate_stats_limit-9-9da67c62d5e3145d450ad7953b33361f
diff --git a/sql/hive/src/test/resources/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/annotate_stats_part-0-2a0c41508e1e70eaedf9de99751c8fa9
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf
rename to sql/hive/src/test/resources/golden/annotate_stats_part-0-2a0c41508e1e70eaedf9de99751c8fa9
diff --git a/sql/hive/src/test/resources/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/annotate_stats_part-1-3c29684bfd2df7439ee0551eb42cfa0
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86
rename to sql/hive/src/test/resources/golden/annotate_stats_part-1-3c29684bfd2df7439ee0551eb42cfa0
diff --git a/sql/hive/src/test/resources/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf b/sql/hive/src/test/resources/golden/annotate_stats_part-10-5ba0295bfe42deb678e59b3a330b14ff
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf
rename to sql/hive/src/test/resources/golden/annotate_stats_part-10-5ba0295bfe42deb678e59b3a330b14ff
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 b/sql/hive/src/test/resources/golden/annotate_stats_part-11-dbdbe2e04c5dad6c8af78b6386b329
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1
rename to sql/hive/src/test/resources/golden/annotate_stats_part-11-dbdbe2e04c5dad6c8af78b6386b329
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/annotate_stats_part-12-2856fb4f4af8932804cb238429d9de6f
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05
rename to sql/hive/src/test/resources/golden/annotate_stats_part-12-2856fb4f4af8932804cb238429d9de6f
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/annotate_stats_part-13-4fa8b0f9fb0202ac0e82fb87538d6445
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0
rename to sql/hive/src/test/resources/golden/annotate_stats_part-13-4fa8b0f9fb0202ac0e82fb87538d6445
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a b/sql/hive/src/test/resources/golden/annotate_stats_part-14-62c557bfb7bf70694a32ebceea329ebd
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a
rename to sql/hive/src/test/resources/golden/annotate_stats_part-14-62c557bfb7bf70694a32ebceea329ebd
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f b/sql/hive/src/test/resources/golden/annotate_stats_part-15-f796cd035603726a5c4ce3e71194822b
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f
rename to sql/hive/src/test/resources/golden/annotate_stats_part-15-f796cd035603726a5c4ce3e71194822b
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 b/sql/hive/src/test/resources/golden/annotate_stats_part-16-45eb5544124364714549f199f9c2b2ac
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301
rename to sql/hive/src/test/resources/golden/annotate_stats_part-16-45eb5544124364714549f199f9c2b2ac
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/annotate_stats_part-17-ad61ebd912b6bef3778c4ff38c0be5ab
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05
rename to sql/hive/src/test/resources/golden/annotate_stats_part-17-ad61ebd912b6bef3778c4ff38c0be5ab
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/annotate_stats_part-18-b9f2dff7c7b57412cea44433ea810fa7
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0
rename to sql/hive/src/test/resources/golden/annotate_stats_part-18-b9f2dff7c7b57412cea44433ea810fa7
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f b/sql/hive/src/test/resources/golden/annotate_stats_part-19-10cab43c2966718bb39e2f22365cd6c1
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f
rename to sql/hive/src/test/resources/golden/annotate_stats_part-19-10cab43c2966718bb39e2f22365cd6c1
diff --git a/sql/hive/src/test/resources/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/annotate_stats_part-2-a4fb8359a2179ec70777aad6366071b7
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076
rename to sql/hive/src/test/resources/golden/annotate_stats_part-2-a4fb8359a2179ec70777aad6366071b7
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a b/sql/hive/src/test/resources/golden/annotate_stats_part-20-51ec5046b50543899ed54c9fc2b885af
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a
rename to sql/hive/src/test/resources/golden/annotate_stats_part-20-51ec5046b50543899ed54c9fc2b885af
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 b/sql/hive/src/test/resources/golden/annotate_stats_part-21-d69cb7b7770b51bc4b99d5d0f74d4662
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5
rename to sql/hive/src/test/resources/golden/annotate_stats_part-21-d69cb7b7770b51bc4b99d5d0f74d4662
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 b/sql/hive/src/test/resources/golden/annotate_stats_part-22-22f430881fbfe92736988c2a4184fe0c
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19
rename to sql/hive/src/test/resources/golden/annotate_stats_part-22-22f430881fbfe92736988c2a4184fe0c
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_part-23-2719ee3f0b8b07f1e5c80f8329b9f87f
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278
rename to sql/hive/src/test/resources/golden/annotate_stats_part-23-2719ee3f0b8b07f1e5c80f8329b9f87f
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/annotate_stats_part-24-1f7bdd816b83f45a6d53d08b6866109f
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09
rename to sql/hive/src/test/resources/golden/annotate_stats_part-24-1f7bdd816b83f45a6d53d08b6866109f
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_part-25-d1599e385e8bad6cd62d564a6fd0536f
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278
rename to sql/hive/src/test/resources/golden/annotate_stats_part-25-d1599e385e8bad6cd62d564a6fd0536f
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/annotate_stats_part-26-ec26bcb2587f4638b364ba494ac373e0
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3
rename to sql/hive/src/test/resources/golden/annotate_stats_part-26-ec26bcb2587f4638b364ba494ac373e0
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/annotate_stats_part-27-7804e2e230a42a46d72546fa2381b423
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451
rename to sql/hive/src/test/resources/golden/annotate_stats_part-27-7804e2e230a42a46d72546fa2381b423
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd b/sql/hive/src/test/resources/golden/annotate_stats_part-28-40f2a1f06d181ef93edf42a591cbf15e
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd
rename to sql/hive/src/test/resources/golden/annotate_stats_part-28-40f2a1f06d181ef93edf42a591cbf15e
diff --git a/sql/hive/src/test/resources/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/annotate_stats_part-3-16367c381d4b189b3640c92511244bfe
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea
rename to sql/hive/src/test/resources/golden/annotate_stats_part-3-16367c381d4b189b3640c92511244bfe
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca b/sql/hive/src/test/resources/golden/annotate_stats_part-4-397e834a25a20b68aed5e87d55ea6174
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca
rename to sql/hive/src/test/resources/golden/annotate_stats_part-4-397e834a25a20b68aed5e87d55ea6174
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c b/sql/hive/src/test/resources/golden/annotate_stats_part-5-cef44682d339a67ba765f854da21f976
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c
rename to sql/hive/src/test/resources/golden/annotate_stats_part-5-cef44682d339a67ba765f854da21f976
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf b/sql/hive/src/test/resources/golden/annotate_stats_part-6-f4263aca1612c05cb08242448191ad05
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf
rename to sql/hive/src/test/resources/golden/annotate_stats_part-6-f4263aca1612c05cb08242448191ad05
diff --git a/sql/hive/src/test/resources/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 b/sql/hive/src/test/resources/golden/annotate_stats_part-7-b222533ddda09918c4f5acc408bf1a02
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1
rename to sql/hive/src/test/resources/golden/annotate_stats_part-7-b222533ddda09918c4f5acc408bf1a02
diff --git a/sql/hive/src/test/resources/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 b/sql/hive/src/test/resources/golden/annotate_stats_part-8-84e3979946de67779a9704a3adc2184f
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31
rename to sql/hive/src/test/resources/golden/annotate_stats_part-8-84e3979946de67779a9704a3adc2184f
diff --git a/sql/hive/src/test/resources/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d b/sql/hive/src/test/resources/golden/annotate_stats_part-9-c6c38e93babafe56cd4f177a17d37b8
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d
rename to sql/hive/src/test/resources/golden/annotate_stats_part-9-c6c38e93babafe56cd4f177a17d37b8
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/annotate_stats_table-0-2a0c41508e1e70eaedf9de99751c8fa9
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b
rename to sql/hive/src/test/resources/golden/annotate_stats_table-0-2a0c41508e1e70eaedf9de99751c8fa9
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/annotate_stats_table-1-3c29684bfd2df7439ee0551eb42cfa0
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f
rename to sql/hive/src/test/resources/golden/annotate_stats_table-1-3c29684bfd2df7439ee0551eb42cfa0
diff --git a/sql/hive/src/test/resources/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/annotate_stats_table-10-a7419af512a6c0b60ec51e556608cdad
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09
rename to sql/hive/src/test/resources/golden/annotate_stats_table-10-a7419af512a6c0b60ec51e556608cdad
diff --git a/sql/hive/src/test/resources/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_table-11-4be11140c2e21b3a172224d103844785
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278
rename to sql/hive/src/test/resources/golden/annotate_stats_table-11-4be11140c2e21b3a172224d103844785
diff --git a/sql/hive/src/test/resources/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/annotate_stats_table-12-84bc7cf67544461e9102820ed410dfb6
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3
rename to sql/hive/src/test/resources/golden/annotate_stats_table-12-84bc7cf67544461e9102820ed410dfb6
diff --git a/sql/hive/src/test/resources/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/annotate_stats_table-13-d2acfdf279aad035b31ed61e87bff6
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451
rename to sql/hive/src/test/resources/golden/annotate_stats_table-13-d2acfdf279aad035b31ed61e87bff6
diff --git a/sql/hive/src/test/resources/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a b/sql/hive/src/test/resources/golden/annotate_stats_table-14-60d1c44b8412c4983265e320ecfb25f0
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a
rename to sql/hive/src/test/resources/golden/annotate_stats_table-14-60d1c44b8412c4983265e320ecfb25f0
diff --git a/sql/hive/src/test/resources/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca b/sql/hive/src/test/resources/golden/annotate_stats_table-15-7b185296b6a0c6ceff2df1f89fecc6b4
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca
rename to sql/hive/src/test/resources/golden/annotate_stats_table-15-7b185296b6a0c6ceff2df1f89fecc6b4
diff --git a/sql/hive/src/test/resources/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c b/sql/hive/src/test/resources/golden/annotate_stats_table-16-7cf8cb7d8d0fe3ea94d7b190a2e5ad3d
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c
rename to sql/hive/src/test/resources/golden/annotate_stats_table-16-7cf8cb7d8d0fe3ea94d7b190a2e5ad3d
diff --git a/sql/hive/src/test/resources/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf b/sql/hive/src/test/resources/golden/annotate_stats_table-17-cd84d629d569a09d139f38dae01e677c
similarity index 100%
rename from sql/hive/src/test/resources/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf
rename to sql/hive/src/test/resources/golden/annotate_stats_table-17-cd84d629d569a09d139f38dae01e677c
diff --git a/sql/hive/src/test/resources/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf b/sql/hive/src/test/resources/golden/annotate_stats_table-18-7012af4092c18075f4f3a00b80f491ee
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf
rename to sql/hive/src/test/resources/golden/annotate_stats_table-18-7012af4092c18075f4f3a00b80f491ee
diff --git a/sql/hive/src/test/resources/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 b/sql/hive/src/test/resources/golden/annotate_stats_table-2-84e7846d50fc15e836c83911ce039871
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4
rename to sql/hive/src/test/resources/golden/annotate_stats_table-2-84e7846d50fc15e836c83911ce039871
diff --git a/sql/hive/src/test/resources/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e b/sql/hive/src/test/resources/golden/annotate_stats_table-3-a598c93d86a646cfa6ea86da54bce9b8
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e
rename to sql/hive/src/test/resources/golden/annotate_stats_table-3-a598c93d86a646cfa6ea86da54bce9b8
diff --git a/sql/hive/src/test/resources/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f b/sql/hive/src/test/resources/golden/annotate_stats_table-4-d12ba848d20d1def95379c490068f050
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f
rename to sql/hive/src/test/resources/golden/annotate_stats_table-4-d12ba848d20d1def95379c490068f050
diff --git a/sql/hive/src/test/resources/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 b/sql/hive/src/test/resources/golden/annotate_stats_table-5-4a8ad142e57c5dce2623e92f5e5ad15a
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35
rename to sql/hive/src/test/resources/golden/annotate_stats_table-5-4a8ad142e57c5dce2623e92f5e5ad15a
diff --git a/sql/hive/src/test/resources/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 b/sql/hive/src/test/resources/golden/annotate_stats_table-6-a59dc1c01d48c82d46a5beab759f618d
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7
rename to sql/hive/src/test/resources/golden/annotate_stats_table-6-a59dc1c01d48c82d46a5beab759f618d
diff --git a/sql/hive/src/test/resources/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b b/sql/hive/src/test/resources/golden/annotate_stats_table-7-6292c27f0c4eaeab2a5e6c317e3afa2e
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b
rename to sql/hive/src/test/resources/golden/annotate_stats_table-7-6292c27f0c4eaeab2a5e6c317e3afa2e
diff --git a/sql/hive/src/test/resources/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c b/sql/hive/src/test/resources/golden/annotate_stats_table-8-43cb040e2fe01904bc52e198fcae9b3d
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c
rename to sql/hive/src/test/resources/golden/annotate_stats_table-8-43cb040e2fe01904bc52e198fcae9b3d
diff --git a/sql/hive/src/test/resources/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 b/sql/hive/src/test/resources/golden/annotate_stats_table-9-eadfdc61d22bd22bbf5a69370908a82e
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588
rename to sql/hive/src/test/resources/golden/annotate_stats_table-9-eadfdc61d22bd22bbf5a69370908a82e
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/annotate_stats_union-0-2a0c41508e1e70eaedf9de99751c8fa9
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6
rename to sql/hive/src/test/resources/golden/annotate_stats_union-0-2a0c41508e1e70eaedf9de99751c8fa9
diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 b/sql/hive/src/test/resources/golden/annotate_stats_union-1-26f82fb6734d5bc6f7159b06c0949178
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403
rename to sql/hive/src/test/resources/golden/annotate_stats_union-1-26f82fb6734d5bc6f7159b06c0949178
diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b b/sql/hive/src/test/resources/golden/annotate_stats_union-10-3aa3eb68d092ea0157c5426a5f2ca3f9
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b
rename to sql/hive/src/test/resources/golden/annotate_stats_union-10-3aa3eb68d092ea0157c5426a5f2ca3f9
diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 b/sql/hive/src/test/resources/golden/annotate_stats_union-11-c1302094d7b4456550826535b529468b
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753
rename to sql/hive/src/test/resources/golden/annotate_stats_union-11-c1302094d7b4456550826535b529468b
diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c b/sql/hive/src/test/resources/golden/annotate_stats_union-12-3af760f960a2e0995d1bc4afef0c5aa0
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c
rename to sql/hive/src/test/resources/golden/annotate_stats_union-12-3af760f960a2e0995d1bc4afef0c5aa0
diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 b/sql/hive/src/test/resources/golden/annotate_stats_union-13-26f82fb6734d5bc6f7159b06c0949178
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3
rename to sql/hive/src/test/resources/golden/annotate_stats_union-13-26f82fb6734d5bc6f7159b06c0949178
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 b/sql/hive/src/test/resources/golden/annotate_stats_union-14-eb11e867a36e2114ef0f0f001e01984c
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0
rename to sql/hive/src/test/resources/golden/annotate_stats_union-14-eb11e867a36e2114ef0f0f001e01984c
diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 b/sql/hive/src/test/resources/golden/annotate_stats_union-15-e0b7cb2226234fbea8ae11ea7d9d84bd
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2
rename to sql/hive/src/test/resources/golden/annotate_stats_union-15-e0b7cb2226234fbea8ae11ea7d9d84bd
diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 b/sql/hive/src/test/resources/golden/annotate_stats_union-16-4252006172a476fbc591eebee49bffa3
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35
rename to sql/hive/src/test/resources/golden/annotate_stats_union-16-4252006172a476fbc591eebee49bffa3
diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b b/sql/hive/src/test/resources/golden/annotate_stats_union-17-a2f3b8a636e46e5df514df46c452855f
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b
rename to sql/hive/src/test/resources/golden/annotate_stats_union-17-a2f3b8a636e46e5df514df46c452855f
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/annotate_stats_union-18-58d55f8c8b2489726232a00254499144
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735
rename to sql/hive/src/test/resources/golden/annotate_stats_union-18-58d55f8c8b2489726232a00254499144
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/annotate_stats_union-19-70c851c23a7ac89f0366b9c26085cc3e
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04
rename to sql/hive/src/test/resources/golden/annotate_stats_union-19-70c851c23a7ac89f0366b9c26085cc3e
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/annotate_stats_union-2-eb11e867a36e2114ef0f0f001e01984c
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15
rename to sql/hive/src/test/resources/golden/annotate_stats_union-2-eb11e867a36e2114ef0f0f001e01984c
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/annotate_stats_union-20-222d1fcce7420950436e00d3a1bba957
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838
rename to sql/hive/src/test/resources/golden/annotate_stats_union-20-222d1fcce7420950436e00d3a1bba957
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/annotate_stats_union-21-a5503df8f50e057415719bcfe4c69f13
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a
rename to sql/hive/src/test/resources/golden/annotate_stats_union-21-a5503df8f50e057415719bcfe4c69f13
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/annotate_stats_union-22-a23089e5a76dc35df04d9936d60c4565
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab
rename to sql/hive/src/test/resources/golden/annotate_stats_union-22-a23089e5a76dc35df04d9936d60c4565
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/annotate_stats_union-3-e0b7cb2226234fbea8ae11ea7d9d84bd
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60
rename to sql/hive/src/test/resources/golden/annotate_stats_union-3-e0b7cb2226234fbea8ae11ea7d9d84bd
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/annotate_stats_union-4-4252006172a476fbc591eebee49bffa3
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152
rename to sql/hive/src/test/resources/golden/annotate_stats_union-4-4252006172a476fbc591eebee49bffa3
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/annotate_stats_union-5-a2f3b8a636e46e5df514df46c452855f
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d
rename to sql/hive/src/test/resources/golden/annotate_stats_union-5-a2f3b8a636e46e5df514df46c452855f
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/annotate_stats_union-6-222d1fcce7420950436e00d3a1bba957
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5
rename to sql/hive/src/test/resources/golden/annotate_stats_union-6-222d1fcce7420950436e00d3a1bba957
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/annotate_stats_union-7-aab1a7b7e1fe4d061a580126d67dfd0a
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735
rename to sql/hive/src/test/resources/golden/annotate_stats_union-7-aab1a7b7e1fe4d061a580126d67dfd0a
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/annotate_stats_union-8-172e0d089bd5bcbaf54775a618d826bb
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04
rename to sql/hive/src/test/resources/golden/annotate_stats_union-8-172e0d089bd5bcbaf54775a618d826bb
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/annotate_stats_union-9-69b422026fce4e2d2cde9a478d6aaa40
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15
rename to sql/hive/src/test/resources/golden/annotate_stats_union-9-69b422026fce4e2d2cde9a478d6aaa40
diff --git a/sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 b/sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb
deleted file mode 100644
index 5cd5fb9874d671fa7bd76288d16c8ed2b3931a2a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb
+++ /dev/null
@@ -1 +0,0 @@
-48479881068
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb
deleted file mode 100644
index 5cd5fb9874d671fa7bd76288d16c8ed2b3931a2a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb
+++ /dev/null
@@ -1 +0,0 @@
-48479881068
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 b/sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8
deleted file mode 100644
index 21b3b13a81191b3745cd0bbe304b41a30ebc073d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8
+++ /dev/null
@@ -1 +0,0 @@
-0	3
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 b/sql/hive/src/test/resources/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73
deleted file mode 100644
index 5e5f6ff96623ffa0277d3e49a6de6d99aa8bb41c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73
+++ /dev/null
@@ -1,9 +0,0 @@
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb
deleted file mode 100644
index 5cd5fb9874d671fa7bd76288d16c8ed2b3931a2a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb
+++ /dev/null
@@ -1 +0,0 @@
-48479881068
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827
deleted file mode 100644
index 69ca68f501ff116aedb9f8f17895a48ab12fb8b2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827
+++ /dev/null
@@ -1,6 +0,0 @@
-0
-0
-0
-10
-20
-30
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827
deleted file mode 100644
index 69ca68f501ff116aedb9f8f17895a48ab12fb8b2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827
+++ /dev/null
@@ -1,6 +0,0 @@
-0
-0
-0
-10
-20
-30
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827
deleted file mode 100644
index 69ca68f501ff116aedb9f8f17895a48ab12fb8b2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827
+++ /dev/null
@@ -1,6 +0,0 @@
-0
-0
-0
-10
-20
-30
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526 b/sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526
deleted file mode 100644
index 18a1a7925ff296b3576e07c429875d4648ddd97f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526
+++ /dev/null
@@ -1 +0,0 @@
-48656137
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a b/sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a
deleted file mode 100644
index 18a1a7925ff296b3576e07c429875d4648ddd97f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a
+++ /dev/null
@@ -1 +0,0 @@
-48656137
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb
deleted file mode 100644
index 5cd5fb9874d671fa7bd76288d16c8ed2b3931a2a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb
+++ /dev/null
@@ -1 +0,0 @@
-48479881068
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb
deleted file mode 100644
index 5cd5fb9874d671fa7bd76288d16c8ed2b3931a2a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb
+++ /dev/null
@@ -1 +0,0 @@
-48479881068
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8
deleted file mode 100644
index 21b3b13a81191b3745cd0bbe304b41a30ebc073d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8
+++ /dev/null
@@ -1 +0,0 @@
-0	3
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73
deleted file mode 100644
index 5e5f6ff96623ffa0277d3e49a6de6d99aa8bb41c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73
+++ /dev/null
@@ -1,9 +0,0 @@
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb
deleted file mode 100644
index 5cd5fb9874d671fa7bd76288d16c8ed2b3931a2a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb
+++ /dev/null
@@ -1 +0,0 @@
-48479881068
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827
deleted file mode 100644
index 69ca68f501ff116aedb9f8f17895a48ab12fb8b2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827
+++ /dev/null
@@ -1,6 +0,0 @@
-0
-0
-0
-10
-20
-30
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827
deleted file mode 100644
index 69ca68f501ff116aedb9f8f17895a48ab12fb8b2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827
+++ /dev/null
@@ -1,6 +0,0 @@
-0
-0
-0
-10
-20
-30
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827
deleted file mode 100644
index 69ca68f501ff116aedb9f8f17895a48ab12fb8b2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827
+++ /dev/null
@@ -1,6 +0,0 @@
-0
-0
-0
-10
-20
-30
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526
deleted file mode 100644
index 18a1a7925ff296b3576e07c429875d4648ddd97f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526
+++ /dev/null
@@ -1 +0,0 @@
-48656137
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a
deleted file mode 100644
index 18a1a7925ff296b3576e07c429875d4648ddd97f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a
+++ /dev/null
@@ -1 +0,0 @@
-48656137
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 b/sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e b/sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e
deleted file mode 100644
index 5cd5fb9874d671fa7bd76288d16c8ed2b3931a2a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e
+++ /dev/null
@@ -1 +0,0 @@
-48479881068
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb
deleted file mode 100644
index 5cd5fb9874d671fa7bd76288d16c8ed2b3931a2a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb
+++ /dev/null
@@ -1 +0,0 @@
-48479881068
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 b/sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8
deleted file mode 100644
index 21b3b13a81191b3745cd0bbe304b41a30ebc073d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8
+++ /dev/null
@@ -1 +0,0 @@
-0	3
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 b/sql/hive/src/test/resources/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73
deleted file mode 100644
index 5e5f6ff96623ffa0277d3e49a6de6d99aa8bb41c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73
+++ /dev/null
@@ -1,9 +0,0 @@
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
-0	val_0	2008-04-08	12	0	val_0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb
deleted file mode 100644
index 5cd5fb9874d671fa7bd76288d16c8ed2b3931a2a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb
+++ /dev/null
@@ -1 +0,0 @@
-48479881068
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba b/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba
+++ b/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c b/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c
index 308fc0924e670a044d307dc8ebe97344bdbe78f7..8e9be7f8c620b8bf64570a7bd69aaac64f700e9c 100644
--- a/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c
+++ b/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c
@@ -1 +1 @@
-34298511120
\ No newline at end of file
+34298511120
diff --git a/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a b/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a
+++ b/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e b/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e
index 16f90efbe50f6e0ddf625bed9f7c0aa07be1277b..d14fbdc94256cdc3ad7f9d7baacd3be4fcf1153b 100644
--- a/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e
+++ b/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e
@@ -1 +1 @@
-101861029915
\ No newline at end of file
+101861029915
diff --git a/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 b/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13
index 16f90efbe50f6e0ddf625bed9f7c0aa07be1277b..d14fbdc94256cdc3ad7f9d7baacd3be4fcf1153b 100644
--- a/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13
+++ b/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13
@@ -1 +1 @@
-101861029915
\ No newline at end of file
+101861029915
diff --git a/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 b/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05
index 69dbf8c3143e99f6ee0216a3c67030d84bcdbd91..faeebdd1eafdf3513e1cfdf2f647cc19d020fd63 100644
--- a/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05
+++ b/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05
@@ -1 +1 @@
--101339664144
\ No newline at end of file
+-101339664144
diff --git a/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 b/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320
index eff107c7ce6bcb1dbf56c2590d4d057c2069531d..caf74830fbd0590053741700a78ccbddf47ce941 100644
--- a/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320
+++ b/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320
@@ -1 +1 @@
--136852761207
\ No newline at end of file
+-136852761207
diff --git a/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 b/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9
index de6c015da2059029c216d6308494c184e336533f..e9710810155153ef8b798250264ba5977368d127 100644
--- a/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9
+++ b/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9
@@ -1 +1 @@
--97676500536
\ No newline at end of file
+-97676500536
diff --git a/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e
+++ b/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880
+++ b/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3
+++ b/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 b/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832
index f1871a4957ddb5ddde17bc7b835061b624297a28..0f27a9bde401c42837b5dee02779111aa703979a 100644
--- a/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832
+++ b/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832
@@ -1 +1 @@
-404554174174
\ No newline at end of file
+404554174174
diff --git a/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 b/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9
index 006e1f82c0a47448c1526f1c93b5bb6a65fa0b09..31d87ddbd13108a370d2c34343de4e2dbac24986 100644
--- a/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9
+++ b/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9
@@ -1 +1 @@
--793937029770
\ No newline at end of file
+-793937029770
diff --git a/sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 b/sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c b/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c
index 006e1f82c0a47448c1526f1c93b5bb6a65fa0b09..31d87ddbd13108a370d2c34343de4e2dbac24986 100644
--- a/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c
+++ b/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c
@@ -1 +1 @@
--793937029770
\ No newline at end of file
+-793937029770
diff --git a/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f b/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f
index 0c9b518e65ece043fef99fe7b51f71290ee9d887..42af6f2e56afcc8d52499f98d64539d35e48b51b 100644
--- a/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f
+++ b/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f
@@ -1 +1 @@
-2358131334
\ No newline at end of file
+2358131334
diff --git a/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f b/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f
index 795166629df40fda82581c862cf2968e2e62587c..069b64b649977444271ad8e5b35164b11393ec2a 100644
--- a/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f
+++ b/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f
@@ -1 +1 @@
-407444119660
\ No newline at end of file
+407444119660
diff --git a/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 b/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185
index 1c958900f50139ea1d226d85fb61f6b5c922b2dd..b8f473bf53aa3ed6bddc79e8607876b571bf5296 100644
--- a/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185
+++ b/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185
@@ -1 +1 @@
-33815990627
\ No newline at end of file
+33815990627
diff --git a/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 b/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52
index b1a6075f768c8de11277df7c9165ffc806b30ef1..6b72a1789f57df6ef26c8be3ed8190d66bb711ac 100644
--- a/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52
+++ b/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52
@@ -1 +1 @@
--24276731469
\ No newline at end of file
+-24276731469
diff --git a/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 b/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2
index b1a6075f768c8de11277df7c9165ffc806b30ef1..6b72a1789f57df6ef26c8be3ed8190d66bb711ac 100644
--- a/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2
+++ b/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2
@@ -1 +1 @@
--24276731469
\ No newline at end of file
+-24276731469
diff --git a/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f b/sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f
index 9672e21fa0323ee3c9a35fe60b37c1abaa0ca092..80c230cf4a09dfa7582adcc24494ac639b270894 100644
--- a/sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f
+++ b/sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f
@@ -497,4 +497,4 @@ NULL	NULL	NULL	NULL	496	val_496
 NULL	NULL	NULL	NULL	497	val_497
 NULL	NULL	NULL	NULL	498	val_498
 NULL	NULL	NULL	NULL	498	val_498
-NULL	NULL	NULL	NULL	498	val_498
\ No newline at end of file
+NULL	NULL	NULL	NULL	498	val_498
diff --git a/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb b/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb
index dba3bca53f72dd2bb752c2d76803e9d26c57a3d5..79162030a2043d97bab2592a8bf0077c0df60ff4 100644
--- a/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb
+++ b/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb
@@ -1 +1 @@
-344337359100
\ No newline at end of file
+344337359100
diff --git a/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 b/sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150
index 5707ed08e7e5406a7505b26df5b80ad66114cfac..13e88f30fc08c5b24e1d6380d831f1da68cb74ec 100644
--- a/sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150
+++ b/sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150
@@ -97,4 +97,4 @@
 9	val_9	5	val_5
 9	val_9	5	val_5
 9	val_9	8	val_8
-9	val_9	9	val_9
\ No newline at end of file
+9	val_9	9	val_9
diff --git a/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 b/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812
+++ b/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa
index 71094ee7360db3e0af9ce2a3483569385a8060e4..16b313fc58f233e2c4b366410d2e1866975b5835 100644
--- a/sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -12,4 +12,4 @@
 311	3
 369	3
 401	5
-406	4
\ No newline at end of file
+406	4
diff --git a/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e b/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e
index dd35c6b71fc80057f07694a1d0e0ee7348383b74..a16667d785bf88c86b8463603095981b0c340f77 100644
--- a/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e
+++ b/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e
@@ -1 +1 @@
-548
\ No newline at end of file
+548
diff --git a/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c b/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c
+++ b/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f b/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f
index 1434bb76ee93f1b3654668066b248aac36932b0e..3afe52bf4b3ae3491f793292191e0a1c3a5f2fab 100644
--- a/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f
+++ b/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f
@@ -1 +1 @@
-344360994461
\ No newline at end of file
+344360994461
diff --git a/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a b/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a
index 1434bb76ee93f1b3654668066b248aac36932b0e..3afe52bf4b3ae3491f793292191e0a1c3a5f2fab 100644
--- a/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a
+++ b/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a
@@ -1 +1 @@
-344360994461
\ No newline at end of file
+344360994461
diff --git a/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d b/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d
index 1434bb76ee93f1b3654668066b248aac36932b0e..3afe52bf4b3ae3491f793292191e0a1c3a5f2fab 100644
--- a/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d
+++ b/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d
@@ -1 +1 @@
-344360994461
\ No newline at end of file
+344360994461
diff --git a/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 b/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73
index 1434bb76ee93f1b3654668066b248aac36932b0e..3afe52bf4b3ae3491f793292191e0a1c3a5f2fab 100644
--- a/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73
+++ b/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73
@@ -1 +1 @@
-344360994461
\ No newline at end of file
+344360994461
diff --git a/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 b/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436
index 1434bb76ee93f1b3654668066b248aac36932b0e..3afe52bf4b3ae3491f793292191e0a1c3a5f2fab 100644
--- a/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436
+++ b/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436
@@ -1 +1 @@
-344360994461
\ No newline at end of file
+344360994461
diff --git a/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 b/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7
index 16f90efbe50f6e0ddf625bed9f7c0aa07be1277b..d14fbdc94256cdc3ad7f9d7baacd3be4fcf1153b 100644
--- a/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7
+++ b/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7
@@ -1 +1 @@
-101861029915
\ No newline at end of file
+101861029915
diff --git a/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 b/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43
index 16f90efbe50f6e0ddf625bed9f7c0aa07be1277b..d14fbdc94256cdc3ad7f9d7baacd3be4fcf1153b 100644
--- a/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43
+++ b/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43
@@ -1 +1 @@
-101861029915
\ No newline at end of file
+101861029915
diff --git a/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 b/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031
index 16f90efbe50f6e0ddf625bed9f7c0aa07be1277b..d14fbdc94256cdc3ad7f9d7baacd3be4fcf1153b 100644
--- a/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031
+++ b/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031
@@ -1 +1 @@
-101861029915
\ No newline at end of file
+101861029915
diff --git a/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 b/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61
index 1434bb76ee93f1b3654668066b248aac36932b0e..3afe52bf4b3ae3491f793292191e0a1c3a5f2fab 100644
--- a/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61
+++ b/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61
@@ -1 +1 @@
-344360994461
\ No newline at end of file
+344360994461
diff --git a/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 b/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2
index 1434bb76ee93f1b3654668066b248aac36932b0e..3afe52bf4b3ae3491f793292191e0a1c3a5f2fab 100644
--- a/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2
+++ b/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2
@@ -1 +1 @@
-344360994461
\ No newline at end of file
+344360994461
diff --git a/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_join32-12-4a7d51ed5c1d98c518ea74f73c6c7d6c
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838
rename to sql/hive/src/test/resources/golden/auto_join32-12-4a7d51ed5c1d98c518ea74f73c6c7d6c
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_join32-13-ee2dcaae78ae900ffce8d19fbadc3735
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a
rename to sql/hive/src/test/resources/golden/auto_join32-13-ee2dcaae78ae900ffce8d19fbadc3735
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_join32-14-7927c2ce644d1ce1de251405c8563e99
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab
rename to sql/hive/src/test/resources/golden/auto_join32-14-7927c2ce644d1ce1de251405c8563e99
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_join32-15-4cd3b51861720ac06c6deb818c83670
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60
rename to sql/hive/src/test/resources/golden/auto_join32-15-4cd3b51861720ac06c6deb818c83670
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_join32-20-693874ea8e06a8b155394ac27af2b1a7
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152
rename to sql/hive/src/test/resources/golden/auto_join32-20-693874ea8e06a8b155394ac27af2b1a7
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_join32-21-bf8a1bb0baaae9fbf1c3aa656f991f42
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d
rename to sql/hive/src/test/resources/golden/auto_join32-21-bf8a1bb0baaae9fbf1c3aa656f991f42
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_join32-22-3d14d63e996851f51a98f987995d8da6
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5
rename to sql/hive/src/test/resources/golden/auto_join32-22-3d14d63e996851f51a98f987995d8da6
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_join32-23-8b183ec2c164b3b530e802ffc880a5fa
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735
rename to sql/hive/src/test/resources/golden/auto_join32-23-8b183ec2c164b3b530e802ffc880a5fa
diff --git a/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed
+++ b/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef b/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef
+++ b/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 b/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04
index f7d1e92d77207040cde3cbcc7bb8a63f9ac282e0..9e503cc0fbeb9007a9330365c69555d864e869b9 100644
--- a/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04
+++ b/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04
@@ -1 +1 @@
-5079148035
\ No newline at end of file
+5079148035
diff --git a/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 b/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04
index e859b7c4ada7bbdbd69781a5761dc23add7b05a8..1b298efae95b93c9e8fad0325bc98a9994632e01 100644
--- a/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04
+++ b/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04
@@ -1 +1 @@
-9766083196
\ No newline at end of file
+9766083196
diff --git a/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 b/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04
index f15ab2fb14eaa59a86a7a8472d0facff416208c3..1e9e1b54b20740563bb3625a9b654cd4039bf6aa 100644
--- a/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04
+++ b/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04
@@ -1 +1 @@
-2607643291
\ No newline at end of file
+2607643291
diff --git a/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 b/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899
index 5f7f06c079d246838b7da39c3d0135dd6113ea5b..5ac39f668df1c65d2290d44d1619d3fa7ca4bd73 100644
--- a/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899
+++ b/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899
@@ -1 +1 @@
--2315698213
\ No newline at end of file
+-2315698213
diff --git a/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f b/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f
index 16f90efbe50f6e0ddf625bed9f7c0aa07be1277b..d14fbdc94256cdc3ad7f9d7baacd3be4fcf1153b 100644
--- a/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f
+++ b/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f
@@ -1 +1 @@
-101861029915
\ No newline at end of file
+101861029915
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/auto_join_filters-2-bee6095f42de6a16708c2f9addc1b9bd
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04
rename to sql/hive/src/test/resources/golden/auto_join_filters-2-bee6095f42de6a16708c2f9addc1b9bd
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/auto_join_filters-31-268d8fb3cb9b04eb269fe7ec40a24dfe
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15
rename to sql/hive/src/test/resources/golden/auto_join_filters-31-268d8fb3cb9b04eb269fe7ec40a24dfe
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a b/sql/hive/src/test/resources/golden/auto_join_filters-32-6dc6866a65c74d69538b776b41b06c16
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a
rename to sql/hive/src/test/resources/golden/auto_join_filters-32-6dc6866a65c74d69538b776b41b06c16
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 b/sql/hive/src/test/resources/golden/auto_join_filters-33-e884480a0f7273d3e2f2de2ba46b855c
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36
rename to sql/hive/src/test/resources/golden/auto_join_filters-33-e884480a0f7273d3e2f2de2ba46b855c
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 b/sql/hive/src/test/resources/golden/auto_join_filters-34-98fd86aea9cacaa82d43c7468109dd33
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237
rename to sql/hive/src/test/resources/golden/auto_join_filters-34-98fd86aea9cacaa82d43c7468109dd33
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 b/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01
index b201b9c4d934830658bfaa038e4cf4a46ea5d010..acf9ebdfa9579c5432e626af6dab884545831c2c 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01
@@ -1 +1 @@
-4542003
\ No newline at end of file
+4542003
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 b/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2
index d365cdf04366c68f15f6d46761cf5dd7732a8fa3..4380aa676ba6735af9bc1a8396d06107c3c3e0a6 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2
@@ -1 +1 @@
-4542038
\ No newline at end of file
+4542038
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c b/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c
index bc7bcdca25bfbfc17fa1522d6b3265791e00ec25..359888de9ce3cea45aee1471963d9b9026184d2b 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c
@@ -1 +1 @@
-4543491
\ No newline at end of file
+4543491
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 b/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80
index b201b9c4d934830658bfaa038e4cf4a46ea5d010..acf9ebdfa9579c5432e626af6dab884545831c2c 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80
@@ -1 +1 @@
-4542003
\ No newline at end of file
+4542003
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 b/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8
index feea6ee0a8e0dff6ab14b084120bd850f2116dfc..3b196ba0b9f870e58a5c2e8be28baea3a5065b4d 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8
@@ -1 +1 @@
-3079923
\ No newline at end of file
+3079923
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 b/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8
index f713b04028bbd41917c8e9c212cc28ff25efbade..dba80cf2f3b4b6210e63b82bff06805549c4b65f 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8
@@ -1 +1 @@
-4509891
\ No newline at end of file
+4509891
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af b/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af
index a94eda6b2c374df886c29165e0f9a2922a372648..7e29fae3a0aa640bbadbc318777418e6756aa240 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af
@@ -1 +1 @@
-3113558
\ No newline at end of file
+3113558
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 b/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621
index feea6ee0a8e0dff6ab14b084120bd850f2116dfc..3b196ba0b9f870e58a5c2e8be28baea3a5065b4d 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621
@@ -1 +1 @@
-3079923
\ No newline at end of file
+3079923
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb b/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb
index 88c5f95e0d838521bc86d13f4008a878f2aee39a..a4231499b4e56605c79bfafc9f95e76366938a0c 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb
@@ -1 +1 @@
-4543526
\ No newline at end of file
+4543526
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 b/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4
index 88c5f95e0d838521bc86d13f4008a878f2aee39a..a4231499b4e56605c79bfafc9f95e76366938a0c 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4
@@ -1 +1 @@
-4543526
\ No newline at end of file
+4543526
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_join_nulls-2-75b1f5331b62fedb7dbbe6ac93a3c83f
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838
rename to sql/hive/src/test/resources/golden/auto_join_nulls-2-75b1f5331b62fedb7dbbe6ac93a3c83f
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d b/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d
index 88c5f95e0d838521bc86d13f4008a878f2aee39a..a4231499b4e56605c79bfafc9f95e76366938a0c 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d
@@ -1 +1 @@
-4543526
\ No newline at end of file
+4543526
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a b/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a
index 88c5f95e0d838521bc86d13f4008a878f2aee39a..a4231499b4e56605c79bfafc9f95e76366938a0c 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a
@@ -1 +1 @@
-4543526
\ No newline at end of file
+4543526
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 b/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3
index f2ec932ce57f4dd570f973d2cd5afea916cda63b..2fa702a938a458866d0e2e4b28ed0d78d8c22f5e 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3
@@ -1 +1 @@
-3112070
\ No newline at end of file
+3112070
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa b/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa
index a94eda6b2c374df886c29165e0f9a2922a372648..7e29fae3a0aa640bbadbc318777418e6756aa240 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa
@@ -1 +1 @@
-3113558
\ No newline at end of file
+3113558
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 b/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984
index f2ec932ce57f4dd570f973d2cd5afea916cda63b..2fa702a938a458866d0e2e4b28ed0d78d8c22f5e 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984
@@ -1 +1 @@
-3112070
\ No newline at end of file
+3112070
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 b/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7
index 4125efd2dd065c5a83e8fd350f7939e59637ecd1..27994c451682a497bd99ef3db5b5c64528844c0c 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7
@@ -1 +1 @@
-13630578
\ No newline at end of file
+13630578
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 b/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562
index 4125efd2dd065c5a83e8fd350f7939e59637ecd1..27994c451682a497bd99ef3db5b5c64528844c0c 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562
@@ -1 +1 @@
-13630578
\ No newline at end of file
+13630578
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc b/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc
index 4125efd2dd065c5a83e8fd350f7939e59637ecd1..27994c451682a497bd99ef3db5b5c64528844c0c 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc
@@ -1 +1 @@
-13630578
\ No newline at end of file
+13630578
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 b/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124
index e877d44372ecbc1180be4c876d6934ad91d90046..476d8eeee571a980bd80338ab8d4a3982e2264e5 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124
@@ -1 +1 @@
-3078400
\ No newline at end of file
+3078400
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 b/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2
index 18be36a9bdb54d559468ad15fc88badf636927d4..935eec97c5601988a72292836f48a4f208499a35 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2
@@ -1 +1 @@
-4509856
\ No newline at end of file
+4509856
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 b/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54
index f2ec932ce57f4dd570f973d2cd5afea916cda63b..2fa702a938a458866d0e2e4b28ed0d78d8c22f5e 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54
@@ -1 +1 @@
-3112070
\ No newline at end of file
+3112070
diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e b/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e
index e877d44372ecbc1180be4c876d6934ad91d90046..476d8eeee571a980bd80338ab8d4a3982e2264e5 100644
--- a/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e
+++ b/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e
@@ -1 +1 @@
-3078400
\ No newline at end of file
+3078400
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_join_reordering_values-1-2bfb628930d072124636d21d82e3b462
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a
rename to sql/hive/src/test/resources/golden/auto_join_reordering_values-1-2bfb628930d072124636d21d82e3b462
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_join_reordering_values-4-11af6838bb9e04152c2f9a7e2044abe0
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab
rename to sql/hive/src/test/resources/golden/auto_join_reordering_values-4-11af6838bb9e04152c2f9a7e2044abe0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_join_reordering_values-7-99fcaa5203ed3debb52c9086028dc8c2
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60
rename to sql/hive/src/test/resources/golden/auto_join_reordering_values-7-99fcaa5203ed3debb52c9086028dc8c2
diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed b/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed
+++ b/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189
index 8fdd954df9831dfd29ceec0d74829b02f3f5d8c3..2bd5a0a98a36cc08ada88b804d3be047e6aa5b8a 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189
@@ -1 +1 @@
-22
\ No newline at end of file
+22
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97
index 62f9457511f879886bb7728c986fe10b0ece6bcb..1e8b314962144c26d5e0e50fd29d2ca327864913 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97
@@ -1 +1 @@
-6
\ No newline at end of file
+6
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5
index 2eafac63a9a984ff7ccdf284713e445d6da2c5a0..2a51623eae15c205f2982ee40f323c0fc5d607a6 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5
@@ -3,4 +3,4 @@
 4	1	1
 5	9	9
 8	1	1
-9	1	1
\ No newline at end of file
+9	1	1
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562
index 8fdd954df9831dfd29ceec0d74829b02f3f5d8c3..2bd5a0a98a36cc08ada88b804d3be047e6aa5b8a 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562
@@ -1 +1 @@
-22
\ No newline at end of file
+22
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce
index 2ebc6516c7df177c819b752dfac7b5ce5064189f..f6b91e0e1f8dddaac700b51aa7a66f29fc135a02 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce
@@ -1 +1 @@
-56
\ No newline at end of file
+56
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e
index 3d2e6576f591f466061942ea4e88bc6ec3125237..f892bae472dffee045a76690a181842d2c5656ce 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e
@@ -19,4 +19,4 @@
 5	val_5
 5	val_5
 8	val_8
-9	val_9
\ No newline at end of file
+9	val_9
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005
index 34d56da29722053c8c471fd3d9cd91378e7252d7..74ff4beddf949a9851348968178c43322275fce6 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005
@@ -19,4 +19,4 @@
 5	val_5	val_5
 5	val_5	val_5
 8	val_8	val_8
-9	val_9	val_9
\ No newline at end of file
+9	val_9	val_9
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e
index 3d2e6576f591f466061942ea4e88bc6ec3125237..f892bae472dffee045a76690a181842d2c5656ce 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e
@@ -19,4 +19,4 @@
 5	val_5
 5	val_5
 8	val_8
-9	val_9
\ No newline at end of file
+9	val_9
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75
index 4a9735f855f96fbbe0a3a921080423139730a84b..ec7496a567609a3261094e21e78810dfb9095b44 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75
@@ -3,4 +3,4 @@
 4	1
 5	9
 8	1
-9	1
\ No newline at end of file
+9	1
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-4e24a5c0c67a137478e4a8be2a081872
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-4e24a5c0c67a137478e4a8be2a081872
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-5cba470fbd02e730781a3b63fd9aa3e2
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-5cba470fbd02e730781a3b63fd9aa3e2
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-337e909605c780d00ad8895686defa06
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-337e909605c780d00ad8895686defa06
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8
index c24b6ae77df02a87472b208f251fad88382a2e55..e522732c77ec94723e739d22f28df549b0231f5f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8
@@ -1 +1 @@
-38
\ No newline at end of file
+38
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-f42438f3f5c266b997686ba846420ebe
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-f42438f3f5c266b997686ba846420ebe
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660
index c24b6ae77df02a87472b208f251fad88382a2e55..e522732c77ec94723e739d22f28df549b0231f5f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660
@@ -1 +1 @@
-38
\ No newline at end of file
+38
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660
index c24b6ae77df02a87472b208f251fad88382a2e55..e522732c77ec94723e739d22f28df549b0231f5f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660
@@ -1 +1 @@
-38
\ No newline at end of file
+38
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-9e58f8a961723c40a5d1f742251a8fa5
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-9e58f8a961723c40a5d1f742251a8fa5
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-d964114ed76536c8e3cacd231340851c
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-d964114ed76536c8e3cacd231340851c
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-e44aff8a860cf3965752d3e1ce725cde
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-e44aff8a860cf3965752d3e1ce725cde
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-ae582a255a02d91674aab500aee79e20
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-ae582a255a02d91674aab500aee79e20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-962264967269db1d5f28a9a6c60dbf1
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-962264967269db1d5f28a9a6c60dbf1
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-10b03ce2526bf180faaec9310cfab290
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-10b03ce2526bf180faaec9310cfab290
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87
index 86ee83a4a26867a79e0a5a8948153cff8f4cf9a4..425151f3a411f5e088d7753e7c8d016303b1b9d1 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87
@@ -1 +1 @@
-40
\ No newline at end of file
+40
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e
index 301160a93062df23030a69f4b5e4d9bf71866ee9..45a4fb75db864000d01701c0f7a51864bd4daabf 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e
@@ -1 +1 @@
-8
\ No newline at end of file
+8
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-4e24a5c0c67a137478e4a8be2a081872
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-4e24a5c0c67a137478e4a8be2a081872
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-5cba470fbd02e730781a3b63fd9aa3e2
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-5cba470fbd02e730781a3b63fd9aa3e2
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-337e909605c780d00ad8895686defa06
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-337e909605c780d00ad8895686defa06
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8
index c24b6ae77df02a87472b208f251fad88382a2e55..e522732c77ec94723e739d22f28df549b0231f5f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8
@@ -1 +1 @@
-38
\ No newline at end of file
+38
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8
index c24b6ae77df02a87472b208f251fad88382a2e55..e522732c77ec94723e739d22f28df549b0231f5f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8
@@ -1 +1 @@
-38
\ No newline at end of file
+38
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-f42438f3f5c266b997686ba846420ebe
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-f42438f3f5c266b997686ba846420ebe
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa
index c24b6ae77df02a87472b208f251fad88382a2e55..e522732c77ec94723e739d22f28df549b0231f5f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa
@@ -1 +1 @@
-38
\ No newline at end of file
+38
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-9e58f8a961723c40a5d1f742251a8fa5
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-9e58f8a961723c40a5d1f742251a8fa5
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-d964114ed76536c8e3cacd231340851c
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-d964114ed76536c8e3cacd231340851c
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-e44aff8a860cf3965752d3e1ce725cde
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-e44aff8a860cf3965752d3e1ce725cde
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-ae582a255a02d91674aab500aee79e20
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-ae582a255a02d91674aab500aee79e20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-962264967269db1d5f28a9a6c60dbf1
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-962264967269db1d5f28a9a6c60dbf1
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-10b03ce2526bf180faaec9310cfab290
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-10b03ce2526bf180faaec9310cfab290
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-4e24a5c0c67a137478e4a8be2a081872
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-4e24a5c0c67a137478e4a8be2a081872
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-5cba470fbd02e730781a3b63fd9aa3e2
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-5cba470fbd02e730781a3b63fd9aa3e2
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-337e909605c780d00ad8895686defa06
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-337e909605c780d00ad8895686defa06
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-97ba394ab6aad2547f399ebbf757a4b6
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-97ba394ab6aad2547f399ebbf757a4b6
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-73ee2d7b62e2aede20ca5de577cd7b7f
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-73ee2d7b62e2aede20ca5de577cd7b7f
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-5fb8b113a91fbdb15eb35fe1a1d1b4f
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-5fb8b113a91fbdb15eb35fe1a1d1b4f
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-f42438f3f5c266b997686ba846420ebe
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-f42438f3f5c266b997686ba846420ebe
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc
index 83be903e06482de431c208dafff94d7f91c6f676..5629a958479c794e0187870f451c49f9fecd7029 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc
@@ -1 +1 @@
-570
\ No newline at end of file
+570
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-9e58f8a961723c40a5d1f742251a8fa5
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-9e58f8a961723c40a5d1f742251a8fa5
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-d964114ed76536c8e3cacd231340851c
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-d964114ed76536c8e3cacd231340851c
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-e44aff8a860cf3965752d3e1ce725cde
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-e44aff8a860cf3965752d3e1ce725cde
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-ae582a255a02d91674aab500aee79e20
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-ae582a255a02d91674aab500aee79e20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-962264967269db1d5f28a9a6c60dbf1
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-962264967269db1d5f28a9a6c60dbf1
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-10b03ce2526bf180faaec9310cfab290
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-10b03ce2526bf180faaec9310cfab290
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2
index 251ff85eda52de765e9376f5a2fc67310988aaf7..1a9097317aa62a2260786d4db2c088d4bb335bb1 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2
@@ -19,4 +19,4 @@
 5	5
 5	5
 8	8
-9	9
\ No newline at end of file
+9	9
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51
index af8f457e934760f1fa04ef84ff78e0f74b571957..225fc24a2290fa7b4b06876bfbd428301e508eae 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51
@@ -19,4 +19,4 @@ val_5	val_5
 val_5	val_5
 val_5	val_5
 val_8	val_8
-val_9	val_9
\ No newline at end of file
+val_9	val_9
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2
index 251ff85eda52de765e9376f5a2fc67310988aaf7..1a9097317aa62a2260786d4db2c088d4bb335bb1 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2
@@ -19,4 +19,4 @@
 5	5
 5	5
 8	8
-9	9
\ No newline at end of file
+9	9
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51
index af8f457e934760f1fa04ef84ff78e0f74b571957..225fc24a2290fa7b4b06876bfbd428301e508eae 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51
@@ -19,4 +19,4 @@ val_5	val_5
 val_5	val_5
 val_5	val_5
 val_8	val_8
-val_9	val_9
\ No newline at end of file
+val_9	val_9
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2
index 251ff85eda52de765e9376f5a2fc67310988aaf7..1a9097317aa62a2260786d4db2c088d4bb335bb1 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2
@@ -19,4 +19,4 @@
 5	5
 5	5
 8	8
-9	9
\ No newline at end of file
+9	9
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51
index af8f457e934760f1fa04ef84ff78e0f74b571957..225fc24a2290fa7b4b06876bfbd428301e508eae 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51
@@ -19,4 +19,4 @@ val_5	val_5
 val_5	val_5
 val_5	val_5
 val_8	val_8
-val_9	val_9
\ No newline at end of file
+val_9	val_9
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a
index 1758dddccea2b3b02d21228a0d06a45a35c0d861..f5c89552bd3e62bfce023a230e90d141f7a46b2f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a
@@ -1 +1 @@
-32
\ No newline at end of file
+32
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd
index bea0d09c49935aac714f5439285e2706b15e1dd9..c92ba56847bf4771e370843f8259974d78b147f0 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd
@@ -1 +1 @@
-207
\ No newline at end of file
+207
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-0-24ca942f094b14b92086305cc125e833
similarity index 100%
rename from sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-0-24ca942f094b14b92086305cc125e833
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-1-16367c381d4b189b3640c92511244bfe
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-1-16367c381d4b189b3640c92511244bfe
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-10-4706d21b17f993cc1cc94be6b7e04c28
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-10-4706d21b17f993cc1cc94be6b7e04c28
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-11-5e81d0b41cc58d8cc151046c7a111411
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-11-5e81d0b41cc58d8cc151046c7a111411
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-12-e8a77ff790bfd6740489fc4374ec6c3d
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-12-e8a77ff790bfd6740489fc4374ec6c3d
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-13-920818d557d2525dabb5c539a80a1bbb
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-13-920818d557d2525dabb5c539a80a1bbb
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-14-455dfeeba27ecf53923db0cbf0aab908
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-14-455dfeeba27ecf53923db0cbf0aab908
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-15-11f98f575685beedc14a88fc47a61041
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-15-11f98f575685beedc14a88fc47a61041
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-16-dff02d7b4c5242434d5e7449bdb67f8b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-16-dff02d7b4c5242434d5e7449bdb67f8b
new file mode 100644
index 0000000000000000000000000000000000000000..2da41004e89142a0f29a26b282b9307567c61e36
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-16-dff02d7b4c5242434d5e7449bdb67f8b
@@ -0,0 +1,24 @@
+0	val_0	val_0	day1	1
+0	val_0	val_0	day1	1
+0	val_0	val_0	day1	1
+0	val_0	val_0	day1	1
+0	val_0	val_0	day1	1
+0	val_0	val_0	day1	1
+169	val_169	val_169	day1	1
+169	val_169	val_169	day1	1
+169	val_169	val_169	day1	1
+169	val_169	val_169	day1	1
+169	val_169	val_169	day1	1
+169	val_169	val_169	day1	1
+169	val_169	val_169	day1	1
+169	val_169	val_169	day1	1
+374	val_374	val_374	day1	1
+374	val_374	val_374	day1	1
+172	val_172	val_172	day1	1
+172	val_172	val_172	day1	1
+172	val_172	val_172	day1	1
+172	val_172	val_172	day1	1
+103	val_103	val_103	day1	1
+103	val_103	val_103	day1	1
+103	val_103	val_103	day1	1
+103	val_103	val_103	day1	1
diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-2-7b4ad215fc2e75c71c6614a2b6322e8e
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-2-7b4ad215fc2e75c71c6614a2b6322e8e
diff --git a/sql/hive/src/test/resources/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-3-365488a703b0640acda73a7d7e6efa06
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-3-365488a703b0640acda73a7d7e6efa06
diff --git a/sql/hive/src/test/resources/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-4-d0ec6d66ff349db09fd455eec149efdb
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-4-d0ec6d66ff349db09fd455eec149efdb
diff --git a/sql/hive/src/test/resources/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-5-3b0f76816be2c1b18a2058027a19bc9f
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-5-3b0f76816be2c1b18a2058027a19bc9f
diff --git a/sql/hive/src/test/resources/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-6-86473a0498e4361e4db0b4a22f2e8571
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-6-86473a0498e4361e4db0b4a22f2e8571
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-7-7e87a31677022b6a1c360922ef74754e
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-7-7e87a31677022b6a1c360922ef74754e
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-8-17d5c9dd1a25e8a54dc9c7444cbe98c
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-8-17d5c9dd1a25e8a54dc9c7444cbe98c
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-9-ae5880516ea2f924cfbaeb919adc86e
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-9-ae5880516ea2f924cfbaeb919adc86e
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-4e24a5c0c67a137478e4a8be2a081872
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-4e24a5c0c67a137478e4a8be2a081872
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660
index c24b6ae77df02a87472b208f251fad88382a2e55..e522732c77ec94723e739d22f28df549b0231f5f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660
@@ -1 +1 @@
-38
\ No newline at end of file
+38
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-f42438f3f5c266b997686ba846420ebe
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-f42438f3f5c266b997686ba846420ebe
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660
index c24b6ae77df02a87472b208f251fad88382a2e55..e522732c77ec94723e739d22f28df549b0231f5f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660
@@ -1 +1 @@
-38
\ No newline at end of file
+38
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-aa6ec7f17b48cf49f02ab7367453ab39
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-aa6ec7f17b48cf49f02ab7367453ab39
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-66b07c93d79ed9958b8427dad16c3ef3
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-66b07c93d79ed9958b8427dad16c3ef3
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-9e58f8a961723c40a5d1f742251a8fa5
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-9e58f8a961723c40a5d1f742251a8fa5
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-d964114ed76536c8e3cacd231340851c
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-d964114ed76536c8e3cacd231340851c
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-962264967269db1d5f28a9a6c60dbf1
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-962264967269db1d5f28a9a6c60dbf1
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-10b03ce2526bf180faaec9310cfab290
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-10b03ce2526bf180faaec9310cfab290
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-4e24a5c0c67a137478e4a8be2a081872
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-4e24a5c0c67a137478e4a8be2a081872
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8
index c24b6ae77df02a87472b208f251fad88382a2e55..e522732c77ec94723e739d22f28df549b0231f5f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8
@@ -1 +1 @@
-38
\ No newline at end of file
+38
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660
index c24b6ae77df02a87472b208f251fad88382a2e55..e522732c77ec94723e739d22f28df549b0231f5f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660
@@ -1 +1 @@
-38
\ No newline at end of file
+38
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-f42438f3f5c266b997686ba846420ebe
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-f42438f3f5c266b997686ba846420ebe
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660
index c24b6ae77df02a87472b208f251fad88382a2e55..e522732c77ec94723e739d22f28df549b0231f5f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660
@@ -1 +1 @@
-38
\ No newline at end of file
+38
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-47a16cf1686c81c5ba76fd92fa5e05a1
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-47a16cf1686c81c5ba76fd92fa5e05a1
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-45b63361c1e5178b69a1531b238c8460
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-45b63361c1e5178b69a1531b238c8460
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-9e58f8a961723c40a5d1f742251a8fa5
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-9e58f8a961723c40a5d1f742251a8fa5
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-d964114ed76536c8e3cacd231340851c
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-d964114ed76536c8e3cacd231340851c
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-e44aff8a860cf3965752d3e1ce725cde
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-e44aff8a860cf3965752d3e1ce725cde
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-ae582a255a02d91674aab500aee79e20
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-ae582a255a02d91674aab500aee79e20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-4e24a5c0c67a137478e4a8be2a081872
similarity index 100%
rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-4e24a5c0c67a137478e4a8be2a081872
diff --git a/sql/hive/src/test/resources/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-9e58f8a961723c40a5d1f742251a8fa5
similarity index 100%
rename from sql/hive/src/test/resources/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-9e58f8a961723c40a5d1f742251a8fa5
diff --git a/sql/hive/src/test/resources/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-d964114ed76536c8e3cacd231340851c
similarity index 100%
rename from sql/hive/src/test/resources/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-d964114ed76536c8e3cacd231340851c
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8
index c24b6ae77df02a87472b208f251fad88382a2e55..e522732c77ec94723e739d22f28df549b0231f5f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8
@@ -1 +1 @@
-38
\ No newline at end of file
+38
diff --git a/sql/hive/src/test/resources/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-f42438f3f5c266b997686ba846420ebe
similarity index 100%
rename from sql/hive/src/test/resources/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-f42438f3f5c266b997686ba846420ebe
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660
index c24b6ae77df02a87472b208f251fad88382a2e55..e522732c77ec94723e739d22f28df549b0231f5f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660
@@ -1 +1 @@
-38
\ No newline at end of file
+38
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660
index c24b6ae77df02a87472b208f251fad88382a2e55..e522732c77ec94723e739d22f28df549b0231f5f 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660
@@ -1 +1 @@
-38
\ No newline at end of file
+38
diff --git a/sql/hive/src/test/resources/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-aa6ec7f17b48cf49f02ab7367453ab39
similarity index 100%
rename from sql/hive/src/test/resources/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-aa6ec7f17b48cf49f02ab7367453ab39
diff --git a/sql/hive/src/test/resources/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-66b07c93d79ed9958b8427dad16c3ef3
similarity index 100%
rename from sql/hive/src/test/resources/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-66b07c93d79ed9958b8427dad16c3ef3
diff --git a/sql/hive/src/test/resources/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-47a16cf1686c81c5ba76fd92fa5e05a1
similarity index 100%
rename from sql/hive/src/test/resources/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-47a16cf1686c81c5ba76fd92fa5e05a1
diff --git a/sql/hive/src/test/resources/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-45b63361c1e5178b69a1531b238c8460
similarity index 100%
rename from sql/hive/src/test/resources/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-45b63361c1e5178b69a1531b238c8460
diff --git a/sql/hive/src/test/resources/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-ecca12a2f377c18c53563a534e9dd5a5
similarity index 100%
rename from sql/hive/src/test/resources/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-ecca12a2f377c18c53563a534e9dd5a5
diff --git a/sql/hive/src/test/resources/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-65930e1b01da720cf296ca3df668b58d
similarity index 100%
rename from sql/hive/src/test/resources/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-65930e1b01da720cf296ca3df668b58d
diff --git a/sql/hive/src/test/resources/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fdbb11a2de2777dfdd916b59764d5c8e
similarity index 100%
rename from sql/hive/src/test/resources/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fdbb11a2de2777dfdd916b59764d5c8e
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8
index dec2bf5d6199c7cd0d84f3dc1e76a73ccc336302..d6b24041cf04154f8f902651969675021f4d93a5 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8
@@ -1 +1 @@
-19
\ No newline at end of file
+19
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660
index dec2bf5d6199c7cd0d84f3dc1e76a73ccc336302..d6b24041cf04154f8f902651969675021f4d93a5 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660
@@ -1 +1 @@
-19
\ No newline at end of file
+19
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-4f56cb50ec6c5cc57974f85d54bcc8ed
similarity index 100%
rename from sql/hive/src/test/resources/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-4f56cb50ec6c5cc57974f85d54bcc8ed
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660
index dec2bf5d6199c7cd0d84f3dc1e76a73ccc336302..d6b24041cf04154f8f902651969675021f4d93a5 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660
@@ -1 +1 @@
-19
\ No newline at end of file
+19
diff --git a/sql/hive/src/test/resources/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-9878d6ab0fe143611c88ffc0602197e3
similarity index 100%
rename from sql/hive/src/test/resources/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-9878d6ab0fe143611c88ffc0602197e3
diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-a576657b9fd1c23aed85a7409887b2fb
similarity index 100%
rename from sql/hive/src/test/resources/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-a576657b9fd1c23aed85a7409887b2fb
diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-350b202868590b5edaed18007fd5cbbe
similarity index 100%
rename from sql/hive/src/test/resources/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-350b202868590b5edaed18007fd5cbbe
diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-1155788c7c133a73c3609e8052accfa5
similarity index 100%
rename from sql/hive/src/test/resources/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-1155788c7c133a73c3609e8052accfa5
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7
index 92c15ec11569f70c649b261e128017ca3e4bd932..f3653aba2d12e0740855ec32c9442e15b74ac1c7 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7
@@ -1 +1 @@
-2654
\ No newline at end of file
+2654
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0
index 92c15ec11569f70c649b261e128017ca3e4bd932..f3653aba2d12e0740855ec32c9442e15b74ac1c7 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0
@@ -1 +1 @@
-2654
\ No newline at end of file
+2654
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186
index 92c15ec11569f70c649b261e128017ca3e4bd932..f3653aba2d12e0740855ec32c9442e15b74ac1c7 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186
@@ -1 +1 @@
-2654
\ No newline at end of file
+2654
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b
index 92c15ec11569f70c649b261e128017ca3e4bd932..f3653aba2d12e0740855ec32c9442e15b74ac1c7 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b
@@ -1 +1 @@
-2654
\ No newline at end of file
+2654
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140
index 92c15ec11569f70c649b261e128017ca3e4bd932..f3653aba2d12e0740855ec32c9442e15b74ac1c7 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140
@@ -1 +1 @@
-2654
\ No newline at end of file
+2654
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0
index 92c15ec11569f70c649b261e128017ca3e4bd932..f3653aba2d12e0740855ec32c9442e15b74ac1c7 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0
@@ -1 +1 @@
-2654
\ No newline at end of file
+2654
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7
index 92c15ec11569f70c649b261e128017ca3e4bd932..f3653aba2d12e0740855ec32c9442e15b74ac1c7 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7
@@ -1 +1 @@
-2654
\ No newline at end of file
+2654
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b
index 92c15ec11569f70c649b261e128017ca3e4bd932..f3653aba2d12e0740855ec32c9442e15b74ac1c7 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b
@@ -1 +1 @@
-2654
\ No newline at end of file
+2654
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140
index 92c15ec11569f70c649b261e128017ca3e4bd932..f3653aba2d12e0740855ec32c9442e15b74ac1c7 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140
@@ -1 +1 @@
-2654
\ No newline at end of file
+2654
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0
index 92c15ec11569f70c649b261e128017ca3e4bd932..f3653aba2d12e0740855ec32c9442e15b74ac1c7 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0
@@ -1 +1 @@
-2654
\ No newline at end of file
+2654
diff --git a/sql/hive/src/test/resources/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-4e24a5c0c67a137478e4a8be2a081872
similarity index 100%
rename from sql/hive/src/test/resources/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-4e24a5c0c67a137478e4a8be2a081872
diff --git a/sql/hive/src/test/resources/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-9e58f8a961723c40a5d1f742251a8fa5
similarity index 100%
rename from sql/hive/src/test/resources/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-9e58f8a961723c40a5d1f742251a8fa5
diff --git a/sql/hive/src/test/resources/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-d964114ed76536c8e3cacd231340851c
similarity index 100%
rename from sql/hive/src/test/resources/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-d964114ed76536c8e3cacd231340851c
diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-962264967269db1d5f28a9a6c60dbf1
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-962264967269db1d5f28a9a6c60dbf1
diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-10b03ce2526bf180faaec9310cfab290
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-10b03ce2526bf180faaec9310cfab290
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-f42438f3f5c266b997686ba846420ebe
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-f42438f3f5c266b997686ba846420ebe
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8
index aa92725341cfdd96ebd08910d54dcb6933dd5be5..dd475631baeff3ce652b97d921c1cc43c917b9e2 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8
@@ -1 +1 @@
-76
\ No newline at end of file
+76
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660
index aa92725341cfdd96ebd08910d54dcb6933dd5be5..dd475631baeff3ce652b97d921c1cc43c917b9e2 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660
@@ -1 +1 @@
-76
\ No newline at end of file
+76
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660
index aa92725341cfdd96ebd08910d54dcb6933dd5be5..dd475631baeff3ce652b97d921c1cc43c917b9e2 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660
@@ -1 +1 @@
-76
\ No newline at end of file
+76
diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-aa6ec7f17b48cf49f02ab7367453ab39
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-aa6ec7f17b48cf49f02ab7367453ab39
diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-66b07c93d79ed9958b8427dad16c3ef3
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-66b07c93d79ed9958b8427dad16c3ef3
diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-47a16cf1686c81c5ba76fd92fa5e05a1
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-47a16cf1686c81c5ba76fd92fa5e05a1
diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-45b63361c1e5178b69a1531b238c8460
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-45b63361c1e5178b69a1531b238c8460
diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-ecca12a2f377c18c53563a534e9dd5a5
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-ecca12a2f377c18c53563a534e9dd5a5
diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-65930e1b01da720cf296ca3df668b58d
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-65930e1b01da720cf296ca3df668b58d
diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-4e24a5c0c67a137478e4a8be2a081872
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-4e24a5c0c67a137478e4a8be2a081872
diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-962264967269db1d5f28a9a6c60dbf1
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-962264967269db1d5f28a9a6c60dbf1
diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-10b03ce2526bf180faaec9310cfab290
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-10b03ce2526bf180faaec9310cfab290
diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-5cba470fbd02e730781a3b63fd9aa3e2
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-5cba470fbd02e730781a3b63fd9aa3e2
diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-337e909605c780d00ad8895686defa06
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-337e909605c780d00ad8895686defa06
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-f42438f3f5c266b997686ba846420ebe
similarity index 100%
rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-f42438f3f5c266b997686ba846420ebe
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8
index aa92725341cfdd96ebd08910d54dcb6933dd5be5..dd475631baeff3ce652b97d921c1cc43c917b9e2 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8
@@ -1 +1 @@
-76
\ No newline at end of file
+76
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660
index aa92725341cfdd96ebd08910d54dcb6933dd5be5..dd475631baeff3ce652b97d921c1cc43c917b9e2 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660
@@ -1 +1 @@
-76
\ No newline at end of file
+76
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660
index aa92725341cfdd96ebd08910d54dcb6933dd5be5..dd475631baeff3ce652b97d921c1cc43c917b9e2 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660
@@ -1 +1 @@
-76
\ No newline at end of file
+76
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-47a16cf1686c81c5ba76fd92fa5e05a1
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-47a16cf1686c81c5ba76fd92fa5e05a1
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-45b63361c1e5178b69a1531b238c8460
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-45b63361c1e5178b69a1531b238c8460
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-9e58f8a961723c40a5d1f742251a8fa5
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-9e58f8a961723c40a5d1f742251a8fa5
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-d964114ed76536c8e3cacd231340851c
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-d964114ed76536c8e3cacd231340851c
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-e44aff8a860cf3965752d3e1ce725cde
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-e44aff8a860cf3965752d3e1ce725cde
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-ae582a255a02d91674aab500aee79e20
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5
rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-ae582a255a02d91674aab500aee79e20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189
index 8fdd954df9831dfd29ceec0d74829b02f3f5d8c3..2bd5a0a98a36cc08ada88b804d3be047e6aa5b8a 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189
@@ -1 +1 @@
-22
\ No newline at end of file
+22
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476
index 4a9735f855f96fbbe0a3a921080423139730a84b..ec7496a567609a3261094e21e78810dfb9095b44 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476
@@ -3,4 +3,4 @@
 4	1
 5	9
 8	1
-9	1
\ No newline at end of file
+9	1
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97
index 62f9457511f879886bb7728c986fe10b0ece6bcb..1e8b314962144c26d5e0e50fd29d2ca327864913 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97
@@ -1 +1 @@
-6
\ No newline at end of file
+6
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5
index 2eafac63a9a984ff7ccdf284713e445d6da2c5a0..2a51623eae15c205f2982ee40f323c0fc5d607a6 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5
@@ -3,4 +3,4 @@
 4	1	1
 5	9	9
 8	1	1
-9	1	1
\ No newline at end of file
+9	1	1
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562
index 8fdd954df9831dfd29ceec0d74829b02f3f5d8c3..2bd5a0a98a36cc08ada88b804d3be047e6aa5b8a 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562
@@ -1 +1 @@
-22
\ No newline at end of file
+22
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce
index 2ebc6516c7df177c819b752dfac7b5ce5064189f..f6b91e0e1f8dddaac700b51aa7a66f29fc135a02 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce
@@ -1 +1 @@
-56
\ No newline at end of file
+56
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189
index 8fdd954df9831dfd29ceec0d74829b02f3f5d8c3..2bd5a0a98a36cc08ada88b804d3be047e6aa5b8a 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189
@@ -1 +1 @@
-22
\ No newline at end of file
+22
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476
index 4a9735f855f96fbbe0a3a921080423139730a84b..ec7496a567609a3261094e21e78810dfb9095b44 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476
@@ -3,4 +3,4 @@
 4	1
 5	9
 8	1
-9	1
\ No newline at end of file
+9	1
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97
index 62f9457511f879886bb7728c986fe10b0ece6bcb..1e8b314962144c26d5e0e50fd29d2ca327864913 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97
@@ -1 +1 @@
-6
\ No newline at end of file
+6
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5
index 2eafac63a9a984ff7ccdf284713e445d6da2c5a0..2a51623eae15c205f2982ee40f323c0fc5d607a6 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5
@@ -3,4 +3,4 @@
 4	1	1
 5	9	9
 8	1	1
-9	1	1
\ No newline at end of file
+9	1	1
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce
index 2ebc6516c7df177c819b752dfac7b5ce5064189f..f6b91e0e1f8dddaac700b51aa7a66f29fc135a02 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce
@@ -1 +1 @@
-56
\ No newline at end of file
+56
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4
deleted file mode 100644
index d0eea8a3ca6613e9eb6cea45bf9352e3c9dd9b6b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4
+++ /dev/null
@@ -1,4 +0,0 @@
-ba_key              	binary              	None                
-ba_val              	binary              	None                
-	 	 
-Detailed Table Information	Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426673, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:null), FieldSchema(name:ba_val, type:binary, comment:null)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426673}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4
deleted file mode 100644
index 0a3c9f383a6eff5a4654a78d73820de25caeb0dd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4
+++ /dev/null
@@ -1,4 +0,0 @@
-ba_key              	binary              	from deserializer   
-ba_val              	binary              	from deserializer   
-	 	 
-Detailed Table Information	Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426674, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:from deserializer), FieldSchema(name:ba_val, type:binary, comment:from deserializer)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=tnachen, last_modified_time=1392426674, transient_lastDdlTime=1392426674}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4
deleted file mode 100644
index 66d33b789f1a730f06883c69993896d67c7be591..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4
+++ /dev/null
@@ -1,4 +0,0 @@
-ba_key              	binary              	None                
-ba_val              	binary              	None                
-	 	 
-Detailed Table Information	Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426676, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:null), FieldSchema(name:ba_val, type:binary, comment:null)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426676}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321 b/sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321
deleted file mode 100644
index dcd1d8643e3cb5e0b9ecf5df66e8909573c5f5f5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321
+++ /dev/null
@@ -1 +0,0 @@
-2	val_2
diff --git a/sql/hive/src/test/resources/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/binary_constant-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c
rename to sql/hive/src/test/resources/golden/binary_constant-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 b/sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8
deleted file mode 100644
index 2e65efe2a145dda7ee51d1741299f848e5bf752e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8
+++ /dev/null
@@ -1 +0,0 @@
-a
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/binary_constant-1-604fde0397444c5c9f1d70e6287de664 b/sql/hive/src/test/resources/golden/binary_constant-1-604fde0397444c5c9f1d70e6287de664
new file mode 100644
index 0000000000000000000000000000000000000000..78981922613b2afb6025042ff6bd878ac1994e85
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/binary_constant-1-604fde0397444c5c9f1d70e6287de664
@@ -0,0 +1 @@
+a
diff --git a/sql/hive/src/test/resources/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 b/sql/hive/src/test/resources/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9
deleted file mode 100644
index c5c8d29fdd13e6dee2330bc99daedf917717dbbb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9
+++ /dev/null
@@ -1,500 +0,0 @@
-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
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4
deleted file mode 100644
index df14abbcc193f1fff0bcca2abdfe9c6012593c59..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4
+++ /dev/null
@@ -1,4 +0,0 @@
-ba_key              	binary              	from deserializer   
-ba_val              	binary              	from deserializer   
-	 	 
-Detailed Table Information	Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426678, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:from deserializer), FieldSchema(name:ba_val, type:binary, comment:from deserializer)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=tnachen, last_modified_time=1392426678, transient_lastDdlTime=1392426678}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4
deleted file mode 100644
index 0c246ae33e56f72cf38279b6d737835bdac725c7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4
+++ /dev/null
@@ -1,4 +0,0 @@
-ba_key              	binary              	from deserializer   
-ba_val              	binary              	from deserializer   
-	 	 
-Detailed Table Information	Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426679, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:null), FieldSchema(name:ba_val, type:binary, comment:null)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426679}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 b/sql/hive/src/test/resources/golden/binarysortable_1-1-aca7ae366662c9698b5d2c01a6cb3948
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4
rename to sql/hive/src/test/resources/golden/binarysortable_1-1-aca7ae366662c9698b5d2c01a6cb3948
diff --git a/sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 b/sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2
index 27687b47813a9f01cf785d42c7ee2ee2d066bb90..d6ee76110e8a96f441d1d7113d60ec21837f6393 100644
--- a/sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2
+++ b/sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2
@@ -7,4 +7,4 @@
 a^@bc^A^B^A^@	1.0
 test^@^@^A^Atest	6.0
 test^@test	4.0
-test^Atest	5.0
\ No newline at end of file
+test^Atest	5.0
diff --git a/sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a b/sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a
index 9bdd310949be818a82328fc58e1fadf001aaa609..b62097939c8332844ecb9a8446082ff1b21c49c9 100644
--- a/sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a
+++ b/sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a
@@ -1 +1 @@
-5	5.0	5.0	5.0	5	true	1
\ No newline at end of file
+5	5.0	5.0	5.0	5	true	1
diff --git a/sql/hive/src/test/resources/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 b/sql/hive/src/test/resources/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98
deleted file mode 100644
index bbe268ea91ddf875832d2cc913f1f6a87ed2cbb1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98
+++ /dev/null
@@ -1 +0,0 @@
-1.0	1.4	1.6	1	1	1	1	1	1	1	1	1	1	1	1	1	1	1	NULL	2147483647	-2147483648	32767	-32768	-128	127	NULL	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 b/sql/hive/src/test/resources/golden/char_varchar_udf-0-4de1b2fa0908a3d856474aae2bc38c08
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3
rename to sql/hive/src/test/resources/golden/char_varchar_udf-0-4de1b2fa0908a3d856474aae2bc38c08
diff --git a/sql/hive/src/test/resources/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 b/sql/hive/src/test/resources/golden/char_varchar_udf-1-5b1e7c580ed5e756d30a4c557af8902
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1
rename to sql/hive/src/test/resources/golden/char_varchar_udf-1-5b1e7c580ed5e756d30a4c557af8902
diff --git a/sql/hive/src/test/resources/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 b/sql/hive/src/test/resources/golden/char_varchar_udf-2-880ce74a83bb2bfb9c9bd584b8874ac
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5
rename to sql/hive/src/test/resources/golden/char_varchar_udf-2-880ce74a83bb2bfb9c9bd584b8874ac
diff --git a/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 b/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5
index 87d0f8dd52b6860df38e25bc26e8933ee51c65e6..ae8b7551d963013cc0f58c79cd215c7a43e33f48 100644
--- a/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5
+++ b/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5
@@ -1 +1 @@
-10	val_10
\ No newline at end of file
+10	val_10
diff --git a/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e b/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e
index 54864d264245d72a878aab8df03d300a8781b293..a9169afab64e70fce88978c83f3ebff739ce2b95 100644
--- a/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e
+++ b/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e
@@ -1 +1 @@
-20	val_20
\ No newline at end of file
+20	val_20
diff --git a/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 b/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7
index 54864d264245d72a878aab8df03d300a8781b293..a9169afab64e70fce88978c83f3ebff739ce2b95 100644
--- a/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7
+++ b/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7
@@ -1 +1 @@
-20	val_20
\ No newline at end of file
+20	val_20
diff --git a/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 b/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0
index 54864d264245d72a878aab8df03d300a8781b293..a9169afab64e70fce88978c83f3ebff739ce2b95 100644
--- a/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0
+++ b/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0
@@ -1 +1 @@
-20	val_20
\ No newline at end of file
+20	val_20
diff --git a/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 b/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65
index 1a018b640eb6fe91a5066f9c41de2ee3a7baaffb..e2ef71e141c46e1c37104669096c3ace3d23fafb 100644
--- a/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65
+++ b/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65
@@ -1 +1 @@
-20	val_20	20
\ No newline at end of file
+20	val_20	20
diff --git a/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 b/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7
index 333a4cf9af12375423e68393b9704d59b0fd1ea5..5b04be3cdf046a6bf0ee8f8a4cd65ae12adeae61 100644
--- a/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7
+++ b/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7
@@ -1 +1 @@
-20	val_20	20	val_20
\ No newline at end of file
+20	val_20	20	val_20
diff --git a/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f b/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f
index 333a4cf9af12375423e68393b9704d59b0fd1ea5..5b04be3cdf046a6bf0ee8f8a4cd65ae12adeae61 100644
--- a/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f
+++ b/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f
@@ -1 +1 @@
-20	val_20	20	val_20
\ No newline at end of file
+20	val_20	20	val_20
diff --git a/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e b/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e
index 1a018b640eb6fe91a5066f9c41de2ee3a7baaffb..e2ef71e141c46e1c37104669096c3ace3d23fafb 100644
--- a/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e
+++ b/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e
@@ -1 +1 @@
-20	val_20	20
\ No newline at end of file
+20	val_20	20
diff --git a/sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a b/sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a
index a79654385b09f56a1c55c50d12b062492c80523a..764cbe5f62b60a4429b356ccb291148aad2c2ec8 100644
--- a/sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a
+++ b/sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a
@@ -495,4 +495,4 @@
 497	val_497
 498	val_498
 498	val_498
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f b/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f
index 54864d264245d72a878aab8df03d300a8781b293..a9169afab64e70fce88978c83f3ebff739ce2b95 100644
--- a/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f
+++ b/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f
@@ -1 +1 @@
-20	val_20
\ No newline at end of file
+20	val_20
diff --git a/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 b/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72
index 54864d264245d72a878aab8df03d300a8781b293..a9169afab64e70fce88978c83f3ebff739ce2b95 100644
--- a/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72
+++ b/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72
@@ -1 +1 @@
-20	val_20
\ No newline at end of file
+20	val_20
diff --git a/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 b/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691
index 54864d264245d72a878aab8df03d300a8781b293..a9169afab64e70fce88978c83f3ebff739ce2b95 100644
--- a/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691
+++ b/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691
@@ -1 +1 @@
-20	val_20
\ No newline at end of file
+20	val_20
diff --git a/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 b/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2
index 54864d264245d72a878aab8df03d300a8781b293..a9169afab64e70fce88978c83f3ebff739ce2b95 100644
--- a/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2
+++ b/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2
@@ -1 +1 @@
-20	val_20
\ No newline at end of file
+20	val_20
diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f b/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f
deleted file mode 100644
index 15e92afeeca276935a406a49f31cf8e16278aff2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f
+++ /dev/null
@@ -1,11 +0,0 @@
-[0,0,0]	["0","0","0"]	{"key_0":"value_0"}	1712634731	record_0
-[1,2,3]	["10","100","1000"]	{"key_1":"value_1"}	465985200	record_1
-[2,4,6]	["20","200","2000"]	{"key_2":"value_2"}	-751827638	record_2
-[3,6,9]	["30","300","3000"]	{"key_3":"value_3"}	477111222	record_3
-[4,8,12]	["40","400","4000"]	{"key_4":"value_4"}	-734328909	record_4
-[5,10,15]	["50","500","5000"]	{"key_5":"value_5"}	-1952710710	record_5
-[6,12,18]	["60","600","6000"]	{"key_6":"value_6"}	1244525190	record_6
-[7,14,21]	["70","700","7000"]	{"key_7":"value_7"}	-1461153973	record_7
-[8,16,24]	["80","800","8000"]	{"key_8":"value_8"}	1638581578	record_8
-[9,18,27]	["90","900","9000"]	{"key_9":"value_9"}	336964413	record_9
-NULL	NULL	NULL	0	NULL
diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 b/sql/hive/src/test/resources/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808
deleted file mode 100644
index 6ad4db1788424419f556b714c804ce01b82a306e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808
+++ /dev/null
@@ -1,58 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME Employee_Part) (TOK_PARTSPEC (TOK_PARTVAL employeeSalary 2000.0))) (TOK_TABCOLNAME employeeID))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        employee_part 
-          TableScan
-            alias: employee_part
-            Select Operator
-              expressions:
-                    expr: employeeid
-                    type: int
-              outputColumnNames: employeeid
-              Group By Operator
-                aggregations:
-                      expr: compute_stats(employeeid, 16)
-                bucketGroup: false
-                mode: hash
-                outputColumnNames: _col0
-                Reduce Output Operator
-                  sort order: 
-                  tag: -1
-                  value expressions:
-                        expr: _col0
-                        type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations:
-                expr: compute_stats(VALUE._col0)
-          bucketGroup: false
-          mode: mergepartial
-          outputColumnNames: _col0
-          Select Operator
-            expressions:
-                  expr: _col0
-                  type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint>
-            outputColumnNames: _col0
-            File Output Operator
-              compressed: false
-              GlobalTableId: 0
-              table:
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-
-  Stage: Stage-1
-    Column Stats Work
-      Column Stats Desc:
-          Columns: employeeID
-          Column Types: int
-          Partition: employeesalary=2000.0
-          Table: Employee_Part
-
diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 b/sql/hive/src/test/resources/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3
deleted file mode 100644
index 91ce2a521cde18fd384541dc34b3fdc91a9b48dd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3
+++ /dev/null
@@ -1,129 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME Employee_Part) (TOK_PARTSPEC (TOK_PARTVAL employeeSalary 2000.0))) (TOK_TABCOLNAME employeeID))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        employee_part 
-          TableScan
-            alias: employee_part
-            GatherStats: false
-            Select Operator
-              expressions:
-                    expr: employeeid
-                    type: int
-              outputColumnNames: employeeid
-              Group By Operator
-                aggregations:
-                      expr: compute_stats(employeeid, 16)
-                bucketGroup: false
-                mode: hash
-                outputColumnNames: _col0
-                Reduce Output Operator
-                  sort order: 
-                  tag: -1
-                  value expressions:
-                        expr: _col0
-                        type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>
-      Path -> Alias:
-        file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=2000.0 [employee_part]
-      Path -> Partition:
-        file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=2000.0 
-          Partition
-            base file name: employeesalary=2000.0
-            input format: org.apache.hadoop.mapred.TextInputFormat
-            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-            partition values:
-              employeesalary 2000.0
-            properties:
-              bucket_count -1
-              columns employeeid,employeename
-              columns.types int:string
-              field.delim |
-              file.inputformat org.apache.hadoop.mapred.TextInputFormat
-              file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=2000.0
-              name default.employee_part
-              numFiles 1
-              numRows 0
-              partition_columns employeesalary
-              rawDataSize 0
-              serialization.ddl struct employee_part { i32 employeeid, string employeename}
-              serialization.format |
-              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              totalSize 105
-              transient_lastDdlTime 1389728706
-            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-          
-              input format: org.apache.hadoop.mapred.TextInputFormat
-              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              properties:
-                bucket_count -1
-                columns employeeid,employeename
-                columns.types int:string
-                field.delim |
-                file.inputformat org.apache.hadoop.mapred.TextInputFormat
-                file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part
-                name default.employee_part
-                numFiles 2
-                numPartitions 2
-                numRows 0
-                partition_columns employeesalary
-                rawDataSize 0
-                serialization.ddl struct employee_part { i32 employeeid, string employeename}
-                serialization.format |
-                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                totalSize 210
-                transient_lastDdlTime 1389728706
-              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              name: default.employee_part
-            name: default.employee_part
-      Truncated Path -> Alias:
-        /employee_part/employeesalary=2000.0 [employee_part]
-      Needs Tagging: false
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations:
-                expr: compute_stats(VALUE._col0)
-          bucketGroup: false
-          mode: mergepartial
-          outputColumnNames: _col0
-          Select Operator
-            expressions:
-                  expr: _col0
-                  type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint>
-            outputColumnNames: _col0
-            File Output Operator
-              compressed: false
-              GlobalTableId: 0
-              directory: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-07_241_106202206012377173-1/-ext-10001
-              NumFilesPerFileSink: 1
-              Stats Publishing Key Prefix: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-07_241_106202206012377173-1/-ext-10001/
-              table:
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  properties:
-                    columns _col0
-                    columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint>
-                    escape.delim \
-                    hive.serialization.extend.nesting.levels true
-                    serialization.format 1
-              TotalFiles: 1
-              GatherStats: false
-              MultiFileSpray: false
-
-  Stage: Stage-1
-    Column Stats Work
-      Column Stats Desc:
-          Columns: employeeID
-          Column Types: int
-          Partition: employeesalary=2000.0
-          Table: Employee_Part
-          Is Table Level Stats: false
-
diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 b/sql/hive/src/test/resources/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1
deleted file mode 100644
index 777024f6946e339baaf0375c9622c34a4a8c7224..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1
+++ /dev/null
@@ -1,58 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME Employee_Part) (TOK_PARTSPEC (TOK_PARTVAL employeeSalary 4000.0))) (TOK_TABCOLNAME employeeID))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        employee_part 
-          TableScan
-            alias: employee_part
-            Select Operator
-              expressions:
-                    expr: employeeid
-                    type: int
-              outputColumnNames: employeeid
-              Group By Operator
-                aggregations:
-                      expr: compute_stats(employeeid, 16)
-                bucketGroup: false
-                mode: hash
-                outputColumnNames: _col0
-                Reduce Output Operator
-                  sort order: 
-                  tag: -1
-                  value expressions:
-                        expr: _col0
-                        type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations:
-                expr: compute_stats(VALUE._col0)
-          bucketGroup: false
-          mode: mergepartial
-          outputColumnNames: _col0
-          Select Operator
-            expressions:
-                  expr: _col0
-                  type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint>
-            outputColumnNames: _col0
-            File Output Operator
-              compressed: false
-              GlobalTableId: 0
-              table:
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-
-  Stage: Stage-1
-    Column Stats Work
-      Column Stats Desc:
-          Columns: employeeID
-          Column Types: int
-          Partition: employeesalary=4000.0
-          Table: Employee_Part
-
diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce b/sql/hive/src/test/resources/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce
deleted file mode 100644
index cd72c7efbf56fd8026c0aa09ddf4af1d5714d086..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce
+++ /dev/null
@@ -1,129 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME Employee_Part) (TOK_PARTSPEC (TOK_PARTVAL employeeSalary 4000.0))) (TOK_TABCOLNAME employeeID))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        employee_part 
-          TableScan
-            alias: employee_part
-            GatherStats: false
-            Select Operator
-              expressions:
-                    expr: employeeid
-                    type: int
-              outputColumnNames: employeeid
-              Group By Operator
-                aggregations:
-                      expr: compute_stats(employeeid, 16)
-                bucketGroup: false
-                mode: hash
-                outputColumnNames: _col0
-                Reduce Output Operator
-                  sort order: 
-                  tag: -1
-                  value expressions:
-                        expr: _col0
-                        type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>
-      Path -> Alias:
-        file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=4000.0 [employee_part]
-      Path -> Partition:
-        file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=4000.0 
-          Partition
-            base file name: employeesalary=4000.0
-            input format: org.apache.hadoop.mapred.TextInputFormat
-            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-            partition values:
-              employeesalary 4000.0
-            properties:
-              bucket_count -1
-              columns employeeid,employeename
-              columns.types int:string
-              field.delim |
-              file.inputformat org.apache.hadoop.mapred.TextInputFormat
-              file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=4000.0
-              name default.employee_part
-              numFiles 1
-              numRows 0
-              partition_columns employeesalary
-              rawDataSize 0
-              serialization.ddl struct employee_part { i32 employeeid, string employeename}
-              serialization.format |
-              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              totalSize 105
-              transient_lastDdlTime 1389728706
-            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-          
-              input format: org.apache.hadoop.mapred.TextInputFormat
-              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              properties:
-                bucket_count -1
-                columns employeeid,employeename
-                columns.types int:string
-                field.delim |
-                file.inputformat org.apache.hadoop.mapred.TextInputFormat
-                file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part
-                name default.employee_part
-                numFiles 2
-                numPartitions 2
-                numRows 0
-                partition_columns employeesalary
-                rawDataSize 0
-                serialization.ddl struct employee_part { i32 employeeid, string employeename}
-                serialization.format |
-                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                totalSize 210
-                transient_lastDdlTime 1389728706
-              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              name: default.employee_part
-            name: default.employee_part
-      Truncated Path -> Alias:
-        /employee_part/employeesalary=4000.0 [employee_part]
-      Needs Tagging: false
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations:
-                expr: compute_stats(VALUE._col0)
-          bucketGroup: false
-          mode: mergepartial
-          outputColumnNames: _col0
-          Select Operator
-            expressions:
-                  expr: _col0
-                  type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint>
-            outputColumnNames: _col0
-            File Output Operator
-              compressed: false
-              GlobalTableId: 0
-              directory: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-24_849_6968895828655634809-1/-ext-10001
-              NumFilesPerFileSink: 1
-              Stats Publishing Key Prefix: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-24_849_6968895828655634809-1/-ext-10001/
-              table:
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  properties:
-                    columns _col0
-                    columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint>
-                    escape.delim \
-                    hive.serialization.extend.nesting.levels true
-                    serialization.format 1
-              TotalFiles: 1
-              GatherStats: false
-              MultiFileSpray: false
-
-  Stage: Stage-1
-    Column Stats Work
-      Column Stats Desc:
-          Columns: employeeID
-          Column Types: int
-          Partition: employeesalary=4000.0
-          Table: Employee_Part
-          Is Table Level Stats: false
-
diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 b/sql/hive/src/test/resources/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95
deleted file mode 100644
index d1e5e7375467d7063f6508426e2da5128ed0da84..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95
+++ /dev/null
@@ -1,73 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME UserVisits_web_text_none)) (TOK_TABCOLNAME sourceIP avgTimeOnSite adRevenue))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        uservisits_web_text_none 
-          TableScan
-            alias: uservisits_web_text_none
-            Select Operator
-              expressions:
-                    expr: sourceip
-                    type: string
-                    expr: avgtimeonsite
-                    type: int
-                    expr: adrevenue
-                    type: float
-              outputColumnNames: sourceip, avgtimeonsite, adrevenue
-              Group By Operator
-                aggregations:
-                      expr: compute_stats(sourceip, 16)
-                      expr: compute_stats(avgtimeonsite, 16)
-                      expr: compute_stats(adrevenue, 16)
-                bucketGroup: false
-                mode: hash
-                outputColumnNames: _col0, _col1, _col2
-                Reduce Output Operator
-                  sort order: 
-                  tag: -1
-                  value expressions:
-                        expr: _col0
-                        type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>
-                        expr: _col1
-                        type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>
-                        expr: _col2
-                        type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:string,numbitvectors:int>
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations:
-                expr: compute_stats(VALUE._col0)
-                expr: compute_stats(VALUE._col1)
-                expr: compute_stats(VALUE._col2)
-          bucketGroup: false
-          mode: mergepartial
-          outputColumnNames: _col0, _col1, _col2
-          Select Operator
-            expressions:
-                  expr: _col0
-                  type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint>
-                  expr: _col1
-                  type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint>
-                  expr: _col2
-                  type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint>
-            outputColumnNames: _col0, _col1, _col2
-            File Output Operator
-              compressed: false
-              GlobalTableId: 0
-              table:
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-
-  Stage: Stage-1
-    Column Stats Work
-      Column Stats Desc:
-          Columns: sourceIP, avgTimeOnSite, adRevenue
-          Column Types: string, int, float
-          Table: UserVisits_web_text_none
-
diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 b/sql/hive/src/test/resources/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770
deleted file mode 100644
index 3f3aa581b43f9e668742eceb9bb543cd42603e2c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770
+++ /dev/null
@@ -1,141 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME UserVisits_web_text_none)) (TOK_TABCOLNAME sourceIP avgTimeOnSite adRevenue))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        uservisits_web_text_none 
-          TableScan
-            alias: uservisits_web_text_none
-            GatherStats: false
-            Select Operator
-              expressions:
-                    expr: sourceip
-                    type: string
-                    expr: avgtimeonsite
-                    type: int
-                    expr: adrevenue
-                    type: float
-              outputColumnNames: sourceip, avgtimeonsite, adrevenue
-              Group By Operator
-                aggregations:
-                      expr: compute_stats(sourceip, 16)
-                      expr: compute_stats(avgtimeonsite, 16)
-                      expr: compute_stats(adrevenue, 16)
-                bucketGroup: false
-                mode: hash
-                outputColumnNames: _col0, _col1, _col2
-                Reduce Output Operator
-                  sort order: 
-                  tag: -1
-                  value expressions:
-                        expr: _col0
-                        type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>
-                        expr: _col1
-                        type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>
-                        expr: _col2
-                        type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:string,numbitvectors:int>
-      Path -> Alias:
-        file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/uservisits_web_text_none [uservisits_web_text_none]
-      Path -> Partition:
-        file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/uservisits_web_text_none 
-          Partition
-            base file name: uservisits_web_text_none
-            input format: org.apache.hadoop.mapred.TextInputFormat
-            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-            properties:
-              bucket_count -1
-              columns sourceip,desturl,visitdate,adrevenue,useragent,ccode,lcode,skeyword,avgtimeonsite
-              columns.types string:string:string:float:string:string:string:string:int
-              field.delim |
-              file.inputformat org.apache.hadoop.mapred.TextInputFormat
-              file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/uservisits_web_text_none
-              name default.uservisits_web_text_none
-              numFiles 1
-              numPartitions 0
-              numRows 0
-              rawDataSize 0
-              serialization.ddl struct uservisits_web_text_none { string sourceip, string desturl, string visitdate, float adrevenue, string useragent, string ccode, string lcode, string skeyword, i32 avgtimeonsite}
-              serialization.format |
-              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              totalSize 7060
-              transient_lastDdlTime 1389728748
-            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-          
-              input format: org.apache.hadoop.mapred.TextInputFormat
-              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              properties:
-                bucket_count -1
-                columns sourceip,desturl,visitdate,adrevenue,useragent,ccode,lcode,skeyword,avgtimeonsite
-                columns.types string:string:string:float:string:string:string:string:int
-                field.delim |
-                file.inputformat org.apache.hadoop.mapred.TextInputFormat
-                file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/uservisits_web_text_none
-                name default.uservisits_web_text_none
-                numFiles 1
-                numPartitions 0
-                numRows 0
-                rawDataSize 0
-                serialization.ddl struct uservisits_web_text_none { string sourceip, string desturl, string visitdate, float adrevenue, string useragent, string ccode, string lcode, string skeyword, i32 avgtimeonsite}
-                serialization.format |
-                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                totalSize 7060
-                transient_lastDdlTime 1389728748
-              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              name: default.uservisits_web_text_none
-            name: default.uservisits_web_text_none
-      Truncated Path -> Alias:
-        /uservisits_web_text_none [uservisits_web_text_none]
-      Needs Tagging: false
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations:
-                expr: compute_stats(VALUE._col0)
-                expr: compute_stats(VALUE._col1)
-                expr: compute_stats(VALUE._col2)
-          bucketGroup: false
-          mode: mergepartial
-          outputColumnNames: _col0, _col1, _col2
-          Select Operator
-            expressions:
-                  expr: _col0
-                  type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint>
-                  expr: _col1
-                  type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint>
-                  expr: _col2
-                  type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint>
-            outputColumnNames: _col0, _col1, _col2
-            File Output Operator
-              compressed: false
-              GlobalTableId: 0
-              directory: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-49_160_8862102294255849057-1/-ext-10001
-              NumFilesPerFileSink: 1
-              Stats Publishing Key Prefix: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-49_160_8862102294255849057-1/-ext-10001/
-              table:
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                  properties:
-                    columns _col0,_col1,_col2
-                    columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint>:struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint>
-                    escape.delim \
-                    hive.serialization.extend.nesting.levels true
-                    serialization.format 1
-              TotalFiles: 1
-              GatherStats: false
-              MultiFileSpray: false
-
-  Stage: Stage-1
-    Column Stats Work
-      Column Stats Desc:
-          Columns: sourceIP, avgTimeOnSite, adRevenue
-          Column Types: string, int, float
-          Table: UserVisits_web_text_none
-          Is Table Level Stats: true
-
diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d b/sql/hive/src/test/resources/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d
deleted file mode 100644
index 4ff444febde63cbad3deedf2d0ff9f135dc327c1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d
+++ /dev/null
@@ -1,89 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME empty_tab)) (TOK_TABCOLNAME a b c d e))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        empty_tab 
-          TableScan
-            alias: empty_tab
-            Select Operator
-              expressions:
-                    expr: a
-                    type: int
-                    expr: b
-                    type: double
-                    expr: c
-                    type: string
-                    expr: d
-                    type: boolean
-                    expr: e
-                    type: binary
-              outputColumnNames: a, b, c, d, e
-              Group By Operator
-                aggregations:
-                      expr: compute_stats(a, 16)
-                      expr: compute_stats(b, 16)
-                      expr: compute_stats(c, 16)
-                      expr: compute_stats(d, 16)
-                      expr: compute_stats(e, 16)
-                bucketGroup: false
-                mode: hash
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                Reduce Output Operator
-                  sort order: 
-                  tag: -1
-                  value expressions:
-                        expr: _col0
-                        type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>
-                        expr: _col1
-                        type: struct<columntype:string,min:double,max:double,countnulls:bigint,bitvector:string,numbitvectors:int>
-                        expr: _col2
-                        type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:string,numbitvectors:int>
-                        expr: _col3
-                        type: struct<columntype:string,counttrues:bigint,countfalses:bigint,countnulls:bigint>
-                        expr: _col4
-                        type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint>
-      Reduce Operator Tree:
-        Group By Operator
-          aggregations:
-                expr: compute_stats(VALUE._col0)
-                expr: compute_stats(VALUE._col1)
-                expr: compute_stats(VALUE._col2)
-                expr: compute_stats(VALUE._col3)
-                expr: compute_stats(VALUE._col4)
-          bucketGroup: false
-          mode: mergepartial
-          outputColumnNames: _col0, _col1, _col2, _col3, _col4
-          Select Operator
-            expressions:
-                  expr: _col0
-                  type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint>
-                  expr: _col1
-                  type: struct<columntype:string,min:double,max:double,countnulls:bigint,numdistinctvalues:bigint>
-                  expr: _col2
-                  type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint>
-                  expr: _col3
-                  type: struct<columntype:string,counttrues:bigint,countfalses:bigint,countnulls:bigint>
-                  expr: _col4
-                  type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint>
-            outputColumnNames: _col0, _col1, _col2, _col3, _col4
-            File Output Operator
-              compressed: false
-              GlobalTableId: 0
-              table:
-                  input format: org.apache.hadoop.mapred.TextInputFormat
-                  output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-
-  Stage: Stage-1
-    Column Stats Work
-      Column Stats Desc:
-          Columns: a, b, c, d, e
-          Column Types: int, double, string, boolean, binary
-          Table: empty_tab
-
diff --git a/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 b/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54
+++ b/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c
+++ b/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf
+++ b/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86
+++ b/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076
+++ b/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea
+++ b/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75 b/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75
+++ b/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 b/sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73
index 8f8e1f4b21fe399326b072398ab22516dd9e38a1..b40eac432935cb8d986c6d517d82397a93201527 100644
--- a/sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73
+++ b/sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73
@@ -497,4 +497,4 @@
 97	val_97
 97	val_97
 98	val_98
-98	val_98
\ No newline at end of file
+98	val_98
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 b/sql/hive/src/test/resources/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748
deleted file mode 100644
index 80fa68b84c17e48d80db3f1d8ae38d1ae685f5b3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748
+++ /dev/null
@@ -1,8 +0,0 @@
-value=2010-04-21 09%3A45%3A00
-value=val_0
-value=val_2
-value=val_4
-value=val_5
-value=val_8
-value=val_9
-value=|
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c b/sql/hive/src/test/resources/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c
deleted file mode 100644
index ff6141674e603d845c824bb2b565283c2888f95c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c
+++ /dev/null
@@ -1,12 +0,0 @@
-0	val_0
-0	val_0
-0	val_0
-11	|
-19	2010-04-21 09:45:00
-2	val_2
-4	val_4
-5	val_5
-5	val_5
-5	val_5
-8	val_8
-9	val_9
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 b/sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31
deleted file mode 100644
index 3cacc0b93c9c9c03a72da624ca28a09ba5c1336f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31
+++ /dev/null
@@ -1 +0,0 @@
-12
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 b/sql/hive/src/test/resources/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97
deleted file mode 100644
index 1a0aa74952afa6ed58ab50433d467b07121a7855..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97
+++ /dev/null
@@ -1,2 +0,0 @@
-2008-04-08	1000
-2008-04-09	1000
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 b/sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 b/sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 b/sql/hive/src/test/resources/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748
deleted file mode 100644
index 80fa68b84c17e48d80db3f1d8ae38d1ae685f5b3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748
+++ /dev/null
@@ -1,8 +0,0 @@
-value=2010-04-21 09%3A45%3A00
-value=val_0
-value=val_2
-value=val_4
-value=val_5
-value=val_8
-value=val_9
-value=|
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c b/sql/hive/src/test/resources/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c
deleted file mode 100644
index ff6141674e603d845c824bb2b565283c2888f95c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c
+++ /dev/null
@@ -1,12 +0,0 @@
-0	val_0
-0	val_0
-0	val_0
-11	|
-19	2010-04-21 09:45:00
-2	val_2
-4	val_4
-5	val_5
-5	val_5
-5	val_5
-8	val_8
-9	val_9
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 b/sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31
deleted file mode 100644
index 3cacc0b93c9c9c03a72da624ca28a09ba5c1336f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31
+++ /dev/null
@@ -1 +0,0 @@
-12
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 b/sql/hive/src/test/resources/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97
deleted file mode 100644
index 1a0aa74952afa6ed58ab50433d467b07121a7855..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97
+++ /dev/null
@@ -1,2 +0,0 @@
-2008-04-08	1000
-2008-04-09	1000
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 b/sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 b/sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 b/sql/hive/src/test/resources/golden/combine3-11-30b8b79049113252bec1cbeac4018a3
deleted file mode 100644
index 80665a4d4c983e761d061a5d9acdb87a776a31d9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine3-11-30b8b79049113252bec1cbeac4018a3
+++ /dev/null
@@ -1,12 +0,0 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-Detailed Partition Information	Partition(values:[2010-08-03, 00], dbName:default, tableName:combine_3_srcpart_seq_rc, createTime:1390898644, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/combine_3_srcpart_seq_rc/ds=2010-08-03/hr=00, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390898644, numRows=500, totalSize=15250, rawDataSize=5312})	
diff --git a/sql/hive/src/test/resources/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b b/sql/hive/src/test/resources/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b
deleted file mode 100644
index 5a87a3aec7cf5b602e8f28bbd6f37c1dd799a97c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b
+++ /dev/null
@@ -1,12 +0,0 @@
-key                 	int                 	from deserializer   
-value               	string              	from deserializer   
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-Detailed Partition Information	Partition(values:[2010-08-03, 001], dbName:default, tableName:combine_3_srcpart_seq_rc, createTime:1390898653, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/combine_3_srcpart_seq_rc/ds=2010-08-03/hr=001, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390898653, numRows=500, totalSize=2202, rawDataSize=4551})	
diff --git a/sql/hive/src/test/resources/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc b/sql/hive/src/test/resources/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc
deleted file mode 100644
index 7b6455db7834b560c48a6753cf6ac8c496b1cc4d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc
+++ /dev/null
@@ -1,30 +0,0 @@
-0	val_0	2010-08-03	00
-0	val_0	2010-08-03	00
-0	val_0	2010-08-03	00
-0	val_0	2010-08-03	001
-0	val_0	2010-08-03	001
-0	val_0	2010-08-03	001
-2	val_2	2010-08-03	00
-2	val_2	2010-08-03	001
-4	val_4	2010-08-03	00
-4	val_4	2010-08-03	001
-5	val_5	2010-08-03	00
-5	val_5	2010-08-03	00
-5	val_5	2010-08-03	00
-5	val_5	2010-08-03	001
-5	val_5	2010-08-03	001
-5	val_5	2010-08-03	001
-8	val_8	2010-08-03	00
-8	val_8	2010-08-03	001
-9	val_9	2010-08-03	00
-9	val_9	2010-08-03	001
-10	val_10	2010-08-03	00
-10	val_10	2010-08-03	001
-11	val_11	2010-08-03	00
-11	val_11	2010-08-03	001
-12	val_12	2010-08-03	00
-12	val_12	2010-08-03	00
-12	val_12	2010-08-03	001
-12	val_12	2010-08-03	001
-15	val_15	2010-08-03	00
-15	val_15	2010-08-03	00
diff --git a/sql/hive/src/test/resources/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 b/sql/hive/src/test/resources/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56
deleted file mode 100644
index d57cb5369e219cd835653dea5e1410712feeef8c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56
+++ /dev/null
@@ -1,30 +0,0 @@
-0	1
-0	1
-0	1
-0	11
-0	11
-0	11
-2	1
-2	11
-4	1
-4	11
-8	1
-8	11
-10	1
-10	11
-12	1
-12	1
-12	11
-12	11
-18	1
-18	1
-18	11
-18	11
-20	1
-20	11
-24	1
-24	1
-24	11
-24	11
-26	1
-26	1
diff --git a/sql/hive/src/test/resources/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af b/sql/hive/src/test/resources/golden/compute_stats_binary-1-44b15c6c9600109e064a5ea5f9c81051
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af
rename to sql/hive/src/test/resources/golden/compute_stats_binary-1-44b15c6c9600109e064a5ea5f9c81051
diff --git a/sql/hive/src/test/resources/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 b/sql/hive/src/test/resources/golden/compute_stats_boolean-1-72ee4bdd5cea69136940dc40e6890e1d
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-10-fb173ed4483e732d367e51f88be793b1
rename to sql/hive/src/test/resources/golden/compute_stats_boolean-1-72ee4bdd5cea69136940dc40e6890e1d
diff --git a/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e b/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e
index dc7b54ad014355b948b93c4c6c5891da053d5fdd..bb95160cb6e07358f54a28a208ae41e69889c97b 100644
--- a/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e
+++ b/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e
@@ -1 +1 @@
-33
\ No newline at end of file
+33
diff --git a/sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d b/sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d
index dd487e6fea3fff5cf1f4a88e5f571b2229a860d0..279805d381a212a17b5c8f27ca38e1118d3bb6f0 100644
--- a/sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d
+++ b/sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d
@@ -1 +1 @@
-{"columntype":"Boolean","counttrues":13,"countfalses":19,"countnulls":1}
\ No newline at end of file
+{"columntype":"Boolean","counttrues":13,"countfalses":19,"countnulls":1}
diff --git a/sql/hive/src/test/resources/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 b/sql/hive/src/test/resources/golden/compute_stats_double-1-8f634b9e334fd58e71844e6283d9794d
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739
rename to sql/hive/src/test/resources/golden/compute_stats_double-1-8f634b9e334fd58e71844e6283d9794d
diff --git a/sql/hive/src/test/resources/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a b/sql/hive/src/test/resources/golden/compute_stats_long-1-9313f166464633b3929707d7ef11d758
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a
rename to sql/hive/src/test/resources/golden/compute_stats_long-1-9313f166464633b3929707d7ef11d758
diff --git a/sql/hive/src/test/resources/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b b/sql/hive/src/test/resources/golden/compute_stats_string-1-3491ef2747a8f34899108d4ae8ebc7eb
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b
rename to sql/hive/src/test/resources/golden/compute_stats_string-1-3491ef2747a8f34899108d4ae8ebc7eb
diff --git a/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 b/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59
index c615f0148ccd106865260335190d7fa00bfdbb14..d35bf9093ca9c4132a69544a0e8de58851741b22 100644
--- a/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59
+++ b/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59
@@ -19,9 +19,9 @@ my_stringset        	struct<>            	from deserializer
 my_enumset          	struct<>            	from deserializer   
 my_structset        	struct<>            	from deserializer   
 optionals           	struct<>            	from deserializer   
-b                   	string              	None                
+b                   	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-b                   	string              	None                
\ No newline at end of file
+b                   	string              	                    
diff --git a/sql/hive/src/test/resources/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f b/sql/hive/src/test/resources/golden/correlationoptimizer1-26-5522db58d123d1bec48b6e71c1b258f3
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-26-5522db58d123d1bec48b6e71c1b258f3
diff --git a/sql/hive/src/test/resources/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd b/sql/hive/src/test/resources/golden/correlationoptimizer1-32-76aad6bc7d7e4a28c33aca1f0ba30e65
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-32-76aad6bc7d7e4a28c33aca1f0ba30e65
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-33-7722bcc896718b584f78cecdab1fdc9f b/sql/hive/src/test/resources/golden/correlationoptimizer1-33-7722bcc896718b584f78cecdab1fdc9f
new file mode 100644
index 0000000000000000000000000000000000000000..fb15947b3d0bb63d0ebfe86f560f3c8c6639326a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-33-7722bcc896718b584f78cecdab1fdc9f
@@ -0,0 +1,16 @@
+NULL	NULL	10
+66	val_66	1
+98	val_98	2
+128	NULL	1
+146	val_146	2
+150	val_150	1
+213	val_213	2
+224	NULL	1
+238	val_238	2
+255	val_255	2
+273	val_273	3
+278	val_278	2
+311	val_311	3
+369	NULL	1
+401	val_401	5
+406	val_406	4
diff --git a/sql/hive/src/test/resources/golden/combine3-22-11025483569617a9f014b5defd71e933 b/sql/hive/src/test/resources/golden/correlationoptimizer1-35-e1d4857548e626bb31d70c096b8d0a95
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-22-11025483569617a9f014b5defd71e933
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-35-e1d4857548e626bb31d70c096b8d0a95
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-36-7722bcc896718b584f78cecdab1fdc9f b/sql/hive/src/test/resources/golden/correlationoptimizer1-36-7722bcc896718b584f78cecdab1fdc9f
new file mode 100644
index 0000000000000000000000000000000000000000..fb15947b3d0bb63d0ebfe86f560f3c8c6639326a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-36-7722bcc896718b584f78cecdab1fdc9f
@@ -0,0 +1,16 @@
+NULL	NULL	10
+66	val_66	1
+98	val_98	2
+128	NULL	1
+146	val_146	2
+150	val_150	1
+213	val_213	2
+224	NULL	1
+238	val_238	2
+255	val_255	2
+273	val_273	3
+278	val_278	2
+311	val_311	3
+369	NULL	1
+401	val_401	5
+406	val_406	4
diff --git a/sql/hive/src/test/resources/golden/combine3-23-4725c48df09565618cbffd05953a5f62 b/sql/hive/src/test/resources/golden/correlationoptimizer1-38-ef6502d6b282c8a6d228bba395b24724
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-23-4725c48df09565618cbffd05953a5f62
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-38-ef6502d6b282c8a6d228bba395b24724
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7 b/sql/hive/src/test/resources/golden/correlationoptimizer1-39-ea87e76dba02a46cb958148333e397b7
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-39-ea87e76dba02a46cb958148333e397b7
diff --git a/sql/hive/src/test/resources/golden/combine3-6-4725c48df09565618cbffd05953a5f62 b/sql/hive/src/test/resources/golden/correlationoptimizer1-41-b79b220859c09354e23b533c105ccbab
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-6-4725c48df09565618cbffd05953a5f62
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-41-b79b220859c09354e23b533c105ccbab
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7 b/sql/hive/src/test/resources/golden/correlationoptimizer1-42-ea87e76dba02a46cb958148333e397b7
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-42-ea87e76dba02a46cb958148333e397b7
diff --git a/sql/hive/src/test/resources/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 b/sql/hive/src/test/resources/golden/correlationoptimizer1-44-638e5300f4c892c2bf27bd91a8f81b64
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-44-638e5300f4c892c2bf27bd91a8f81b64
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad b/sql/hive/src/test/resources/golden/correlationoptimizer1-45-66010469a9cdb66851da9a727ef9fdad
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-45-66010469a9cdb66851da9a727ef9fdad
diff --git a/sql/hive/src/test/resources/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b b/sql/hive/src/test/resources/golden/correlationoptimizer1-47-3514c74c7f68f2d70cc6d51ac46c20
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-47-3514c74c7f68f2d70cc6d51ac46c20
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad b/sql/hive/src/test/resources/golden/correlationoptimizer1-48-66010469a9cdb66851da9a727ef9fdad
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-48-66010469a9cdb66851da9a727ef9fdad
diff --git a/sql/hive/src/test/resources/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 b/sql/hive/src/test/resources/golden/correlationoptimizer1-49-b9d963d24994c47c3776dda6f7d3881f
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape2-3-1774adb1085f4ee6782a8dac0735399
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-49-b9d963d24994c47c3776dda6f7d3881f
diff --git a/sql/hive/src/test/resources/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 b/sql/hive/src/test/resources/golden/correlationoptimizer1-50-7490df6719cd7e47aa08dbcbc3266a92
similarity index 100%
rename from sql/hive/src/test/resources/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-50-7490df6719cd7e47aa08dbcbc3266a92
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba b/sql/hive/src/test/resources/golden/correlationoptimizer1-51-e71195e7d9f557e2abc7f03462d22dba
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-51-e71195e7d9f557e2abc7f03462d22dba
diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/correlationoptimizer1-52-777edd9d575f3480ca6cebe4be57b1f6
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-52-777edd9d575f3480ca6cebe4be57b1f6
diff --git a/sql/hive/src/test/resources/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 b/sql/hive/src/test/resources/golden/correlationoptimizer1-53-73da9fe2b0c2ee26c021ec3f2fa27272
similarity index 100%
rename from sql/hive/src/test/resources/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-53-73da9fe2b0c2ee26c021ec3f2fa27272
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba b/sql/hive/src/test/resources/golden/correlationoptimizer1-54-e71195e7d9f557e2abc7f03462d22dba
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-54-e71195e7d9f557e2abc7f03462d22dba
diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/correlationoptimizer1-55-b1e2ade89ae898650f0be4f796d8947b
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-55-b1e2ade89ae898650f0be4f796d8947b
diff --git a/sql/hive/src/test/resources/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 b/sql/hive/src/test/resources/golden/correlationoptimizer1-57-fcf9bcb522f542637ccdea863b408448
similarity index 100%
rename from sql/hive/src/test/resources/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-57-fcf9bcb522f542637ccdea863b408448
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-52-3070366869308907e54797927805603 b/sql/hive/src/test/resources/golden/correlationoptimizer1-58-3070366869308907e54797927805603
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-52-3070366869308907e54797927805603
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-58-3070366869308907e54797927805603
diff --git a/sql/hive/src/test/resources/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 b/sql/hive/src/test/resources/golden/correlationoptimizer1-60-dad56e1f06c808b29e5dc8fb0c49efb2
similarity index 100%
rename from sql/hive/src/test/resources/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-60-dad56e1f06c808b29e5dc8fb0c49efb2
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-55-3070366869308907e54797927805603 b/sql/hive/src/test/resources/golden/correlationoptimizer1-61-3070366869308907e54797927805603
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-55-3070366869308907e54797927805603
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-61-3070366869308907e54797927805603
diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/correlationoptimizer1-62-b9d963d24994c47c3776dda6f7d3881f
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-62-b9d963d24994c47c3776dda6f7d3881f
diff --git a/sql/hive/src/test/resources/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 b/sql/hive/src/test/resources/golden/correlationoptimizer1-63-3cd3fbbbd8ee5c274fe3d6a45126cef4
similarity index 100%
rename from sql/hive/src/test/resources/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-63-3cd3fbbbd8ee5c274fe3d6a45126cef4
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548 b/sql/hive/src/test/resources/golden/correlationoptimizer1-64-a6bba6d9b422adb386b35c62cecb548
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-64-a6bba6d9b422adb386b35c62cecb548
diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/correlationoptimizer1-65-777edd9d575f3480ca6cebe4be57b1f6
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-65-777edd9d575f3480ca6cebe4be57b1f6
diff --git a/sql/hive/src/test/resources/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f b/sql/hive/src/test/resources/golden/correlationoptimizer1-66-d6bbaf0d40010159095e4cac025c50c5
similarity index 100%
rename from sql/hive/src/test/resources/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-66-d6bbaf0d40010159095e4cac025c50c5
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548 b/sql/hive/src/test/resources/golden/correlationoptimizer1-67-a6bba6d9b422adb386b35c62cecb548
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548
rename to sql/hive/src/test/resources/golden/correlationoptimizer1-67-a6bba6d9b422adb386b35c62cecb548
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6
index 0b1ee37d7831c78c3f66457cba3f586863d3fa57..6eebe80953bf026ab6bb7c98a6624d1c2833d94a 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6
@@ -34,4 +34,4 @@
 406	1	406
 406	1	406
 406	1	406
-406	1	406
\ No newline at end of file
+406	1	406
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 b/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6
index 0b1ee37d7831c78c3f66457cba3f586863d3fa57..6eebe80953bf026ab6bb7c98a6624d1c2833d94a 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6
@@ -34,4 +34,4 @@
 406	1	406
 406	1	406
 406	1	406
-406	1	406
\ No newline at end of file
+406	1	406
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 b/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06
index 6c6dc691bbff27b080a0954ece6a76193bdda7bb..e7c8cc75a0d6c3c307b538cd6a9c0a866ea29938 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06
@@ -1 +1 @@
-79136	500	3556	15
\ No newline at end of file
+79136	500	3556	15
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 b/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538
index f4bb720dfd7f82d20d3477b6b388adc5e6c71156..96d1ad9dd0559f07c47f33657abe7a1b7be623e0 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538
@@ -1 +1 @@
-3556	37	3556	25
\ No newline at end of file
+3556	37	3556	25
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 b/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538
index f4bb720dfd7f82d20d3477b6b388adc5e6c71156..96d1ad9dd0559f07c47f33657abe7a1b7be623e0 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538
@@ -1 +1 @@
-3556	37	3556	25
\ No newline at end of file
+3556	37	3556	25
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e b/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e
index 4acbb60e8166194d301c3fa52a0b7e52359601b5..716f95a30304bd654397163f083e9676dae11057 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e
@@ -1 +1 @@
-79136	500	3556	25
\ No newline at end of file
+79136	500	3556	25
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e b/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e
index 4acbb60e8166194d301c3fa52a0b7e52359601b5..716f95a30304bd654397163f083e9676dae11057 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e
@@ -1 +1 @@
-79136	500	3556	25
\ No newline at end of file
+79136	500	3556	25
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 b/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104
index a95fce80fd7b46d884b71931c589e5fbcfeeeb86..3821ee1926f176eb9aa215cc2f2e14246cdef154 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104
@@ -1 +1 @@
-79136	310
\ No newline at end of file
+79136	310
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d b/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d
index c6243d7056353601c2bcf5b8c6e16481fa146656..7e1b7f7408e2ddaee8bea2143a8a2e3567343d44 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d
@@ -1 +1 @@
-3556	37	3556	15
\ No newline at end of file
+3556	37	3556	15
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 b/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104
index a95fce80fd7b46d884b71931c589e5fbcfeeeb86..3821ee1926f176eb9aa215cc2f2e14246cdef154 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104
@@ -1 +1 @@
-79136	310
\ No newline at end of file
+79136	310
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b b/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b
index 556b77ecfc9ebf7f5f4fcfe2fc83f2c3b9074a73..a8707661c9399fba5eaa97440e0c5ed24f4431f5 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b
@@ -1 +1 @@
-9992	3531902962	9992	37
\ No newline at end of file
+9992	3531902962	9992	37
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b b/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b
index 556b77ecfc9ebf7f5f4fcfe2fc83f2c3b9074a73..a8707661c9399fba5eaa97440e0c5ed24f4431f5 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b
@@ -1 +1 @@
-9992	3531902962	9992	37
\ No newline at end of file
+9992	3531902962	9992	37
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d b/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d
index c6243d7056353601c2bcf5b8c6e16481fa146656..7e1b7f7408e2ddaee8bea2143a8a2e3567343d44 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d
@@ -1 +1 @@
-3556	37	3556	15
\ No newline at end of file
+3556	37	3556	15
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 b/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06
index 6c6dc691bbff27b080a0954ece6a76193bdda7bb..e7c8cc75a0d6c3c307b538cd6a9c0a866ea29938 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06
@@ -1 +1 @@
-79136	500	3556	15
\ No newline at end of file
+79136	500	3556	15
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 b/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325
index a1a6cbb91955e21c76286dcdf0694cbd12f82fcd..9ef7747157bddf163df450db05fb3d64fe160d6f 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325
@@ -1 +1 @@
-9992	107	3531902962
\ No newline at end of file
+9992	107	3531902962
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 b/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5
index a1a6cbb91955e21c76286dcdf0694cbd12f82fcd..9ef7747157bddf163df450db05fb3d64fe160d6f 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5
@@ -1 +1 @@
-9992	107	3531902962
\ No newline at end of file
+9992	107	3531902962
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 b/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5
index a1a6cbb91955e21c76286dcdf0694cbd12f82fcd..9ef7747157bddf163df450db05fb3d64fe160d6f 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5
@@ -1 +1 @@
-9992	107	3531902962
\ No newline at end of file
+9992	107	3531902962
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 b/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5
index a1a6cbb91955e21c76286dcdf0694cbd12f82fcd..9ef7747157bddf163df450db05fb3d64fe160d6f 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5
@@ -1 +1 @@
-9992	107	3531902962
\ No newline at end of file
+9992	107	3531902962
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 b/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325
index a1a6cbb91955e21c76286dcdf0694cbd12f82fcd..9ef7747157bddf163df450db05fb3d64fe160d6f 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325
@@ -1 +1 @@
-9992	107	3531902962
\ No newline at end of file
+9992	107	3531902962
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 b/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325
index a1a6cbb91955e21c76286dcdf0694cbd12f82fcd..9ef7747157bddf163df450db05fb3d64fe160d6f 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325
@@ -1 +1 @@
-9992	107	3531902962
\ No newline at end of file
+9992	107	3531902962
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-26-8bcdcc5f01508f576d7bd6422c939225 b/sql/hive/src/test/resources/golden/correlationoptimizer4-1-c0f14def6a135cc50cba364e810ce28e
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-26-8bcdcc5f01508f576d7bd6422c939225
rename to sql/hive/src/test/resources/golden/correlationoptimizer4-1-c0f14def6a135cc50cba364e810ce28e
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e b/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e
index 14e309fdcad892bf2bc79d7dd40230bfc4b26243..281e236e8a80d29e20e157792b1e062a4b361cc5 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e
@@ -1 +1 @@
-13	10
\ No newline at end of file
+13	10
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e b/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e
index 14e309fdcad892bf2bc79d7dd40230bfc4b26243..281e236e8a80d29e20e157792b1e062a4b361cc5 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e
@@ -1 +1 @@
-13	10
\ No newline at end of file
+13	10
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 b/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208
index 1b8c137073e375e9f304c717dc9c1e6dcfb6040d..1ed2737b0d1fb50dc92fa5307fe0903ab4ff2a52 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208
@@ -1 +1 @@
-22	12
\ No newline at end of file
+22	12
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 b/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208
index 1b8c137073e375e9f304c717dc9c1e6dcfb6040d..1ed2737b0d1fb50dc92fa5307fe0903ab4ff2a52 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208
@@ -1 +1 @@
-22	12
\ No newline at end of file
+22	12
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b b/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b
index e6c95e600a2c06a8b8108ea84f9802305f119ace..35b3da1e4da5c71bfb24b5f2f4cd087666cea537 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b
@@ -1 +1 @@
-13	12
\ No newline at end of file
+13	12
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf b/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf
index 0248cc90cb2c6ab6a9c42a528ba933415aac02b8..7b6dfccea7a0ca40c645a82bb308b167caa08ad1 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf
@@ -1 +1 @@
-21	12
\ No newline at end of file
+21	12
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-32-ef6502d6b282c8a6d228bba395b24724 b/sql/hive/src/test/resources/golden/correlationoptimizer4-3-ade68a23d7b1a4f328623bb5a0f07488
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-32-ef6502d6b282c8a6d228bba395b24724
rename to sql/hive/src/test/resources/golden/correlationoptimizer4-3-ade68a23d7b1a4f328623bb5a0f07488
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf b/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf
index 0248cc90cb2c6ab6a9c42a528ba933415aac02b8..7b6dfccea7a0ca40c645a82bb308b167caa08ad1 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf
@@ -1 +1 @@
-21	12
\ No newline at end of file
+21	12
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 b/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8
index 0248cc90cb2c6ab6a9c42a528ba933415aac02b8..7b6dfccea7a0ca40c645a82bb308b167caa08ad1 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8
@@ -1 +1 @@
-21	12
\ No newline at end of file
+21	12
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 b/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2
index fd3a42ebe67e7fe67c5d84651cda87716cad3d30..60ee3e8737989c3eecda56dd72fcd9a5043fb8ff 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2
@@ -1 +1 @@
-21	14
\ No newline at end of file
+21	14
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 b/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2
index fd3a42ebe67e7fe67c5d84651cda87716cad3d30..60ee3e8737989c3eecda56dd72fcd9a5043fb8ff 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2
@@ -1 +1 @@
-21	14
\ No newline at end of file
+21	14
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-35-b79b220859c09354e23b533c105ccbab b/sql/hive/src/test/resources/golden/correlationoptimizer4-5-b5777cff7c522c4b527f77988e7f6bf1
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-35-b79b220859c09354e23b533c105ccbab
rename to sql/hive/src/test/resources/golden/correlationoptimizer4-5-b5777cff7c522c4b527f77988e7f6bf1
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e b/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e
index 14e309fdcad892bf2bc79d7dd40230bfc4b26243..281e236e8a80d29e20e157792b1e062a4b361cc5 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e
@@ -1 +1 @@
-13	10
\ No newline at end of file
+13	10
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 b/sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21
index 19b8a2aea8f64a64292ae66fd0689b51ba536908..8f9dae31cc51c5e6b35c5cab2e2a7d9a19f2229c 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21
@@ -12,4 +12,4 @@
 311	1	311	9
 369	1	369	9
 401	1	401	25
-406	1	406	16
\ No newline at end of file
+406	1	406	16
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db b/sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db
index c4a418f59625b8b6f2959f98aa6dfa6fa4e3af3e..26151f7b6d0f1135176ec2c51464e553405141bf 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db
@@ -12,4 +12,4 @@
 311	311	3
 369	369	3
 401	401	5
-406	406	4
\ No newline at end of file
+406	406	4
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db b/sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db
index c4a418f59625b8b6f2959f98aa6dfa6fa4e3af3e..26151f7b6d0f1135176ec2c51464e553405141bf 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db
@@ -12,4 +12,4 @@
 311	311	3
 369	369	3
 401	401	5
-406	406	4
\ No newline at end of file
+406	406	4
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de b/sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de
index 9c8189500649e29bdf582d5c5446189244c44363..c7d10af90e3532861341160eb5dd05384c47cd04 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de
@@ -12,4 +12,4 @@
 311	311	9
 369	369	9
 401	401	25
-406	406	16
\ No newline at end of file
+406	406	16
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de b/sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de
index 9c8189500649e29bdf582d5c5446189244c44363..c7d10af90e3532861341160eb5dd05384c47cd04 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de
@@ -12,4 +12,4 @@
 311	311	9
 369	369	9
 401	401	25
-406	406	16
\ No newline at end of file
+406	406	16
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6
index 0b1ee37d7831c78c3f66457cba3f586863d3fa57..6eebe80953bf026ab6bb7c98a6624d1c2833d94a 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6
@@ -34,4 +34,4 @@
 406	1	406
 406	1	406
 406	1	406
-406	1	406
\ No newline at end of file
+406	1	406
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 b/sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21
index 19b8a2aea8f64a64292ae66fd0689b51ba536908..8f9dae31cc51c5e6b35c5cab2e2a7d9a19f2229c 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21
@@ -12,4 +12,4 @@
 311	1	311	9
 369	1	369	9
 401	1	401	25
-406	1	406	16
\ No newline at end of file
+406	1	406	16
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6
index 0b1ee37d7831c78c3f66457cba3f586863d3fa57..6eebe80953bf026ab6bb7c98a6624d1c2833d94a 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6
@@ -34,4 +34,4 @@
 406	1	406
 406	1	406
 406	1	406
-406	1	406
\ No newline at end of file
+406	1	406
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 b/sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560
index 7c591d6146fd6eee398695771bc62ef7a82f89e3..4e3bbb2779bf34646a59852c184879bb1f5334af 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560
@@ -34,4 +34,4 @@
 406	406	16
 406	406	16
 406	406	16
-406	406	16
\ No newline at end of file
+406	406	16
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 b/sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560
index 7c591d6146fd6eee398695771bc62ef7a82f89e3..4e3bbb2779bf34646a59852c184879bb1f5334af 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560
@@ -34,4 +34,4 @@
 406	406	16
 406	406	16
 406	406	16
-406	406	16
\ No newline at end of file
+406	406	16
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 b/sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8
index 7c591d6146fd6eee398695771bc62ef7a82f89e3..4e3bbb2779bf34646a59852c184879bb1f5334af 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8
@@ -34,4 +34,4 @@
 406	406	16
 406	406	16
 406	406	16
-406	406	16
\ No newline at end of file
+406	406	16
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 b/sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8
index 7c591d6146fd6eee398695771bc62ef7a82f89e3..4e3bbb2779bf34646a59852c184879bb1f5334af 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8
@@ -34,4 +34,4 @@
 406	406	16
 406	406	16
 406	406	16
-406	406	16
\ No newline at end of file
+406	406	16
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 b/sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0
index 8338433cd5a2720c253525edb49d1f5fa959be96..917ab36da2fbbfc44ed35bffded4ce7cc4f5d67a 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0
@@ -34,4 +34,4 @@
 406	4	1	406	val_406
 406	4	1	406	val_406
 406	4	1	406	val_406
-406	4	1	406	val_406
\ No newline at end of file
+406	4	1	406	val_406
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 b/sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0
index 8338433cd5a2720c253525edb49d1f5fa959be96..917ab36da2fbbfc44ed35bffded4ce7cc4f5d67a 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0
@@ -34,4 +34,4 @@
 406	4	1	406	val_406
 406	4	1	406	val_406
 406	4	1	406	val_406
-406	4	1	406	val_406
\ No newline at end of file
+406	4	1	406	val_406
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d b/sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d
index 8b1d3cd388fa253c4fd899b66dec3189c663849d..a6942feae66e51a174308cfc9c47f19c1fb16157 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d
@@ -12,4 +12,4 @@
 311	1	311	val_311	9
 369	1	369	val_369	9
 401	1	401	val_401	25
-406	1	406	val_406	16
\ No newline at end of file
+406	1	406	val_406	16
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d b/sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d
index 8b1d3cd388fa253c4fd899b66dec3189c663849d..a6942feae66e51a174308cfc9c47f19c1fb16157 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d
@@ -12,4 +12,4 @@
 311	1	311	val_311	9
 369	1	369	val_369	9
 401	1	401	val_401	25
-406	1	406	val_406	16
\ No newline at end of file
+406	1	406	val_406	16
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d b/sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d
index 8b1d3cd388fa253c4fd899b66dec3189c663849d..a6942feae66e51a174308cfc9c47f19c1fb16157 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d
@@ -12,4 +12,4 @@
 311	1	311	val_311	9
 369	1	369	val_369	9
 401	1	401	val_401	25
-406	1	406	val_406	16
\ No newline at end of file
+406	1	406	val_406	16
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 b/sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21
index 19b8a2aea8f64a64292ae66fd0689b51ba536908..8f9dae31cc51c5e6b35c5cab2e2a7d9a19f2229c 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21
@@ -12,4 +12,4 @@
 311	1	311	9
 369	1	369	9
 401	1	401	25
-406	1	406	16
\ No newline at end of file
+406	1	406	16
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42
index 747b650237b5379c16a2704d73c082578edf7f9f..1a82dfc1535657d9e15a3ae35d300591daf64e39 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42
@@ -12,4 +12,4 @@
 311	3	311	val_311
 369	3	369	
 401	5	401	val_401
-406	4	406	val_406
\ No newline at end of file
+406	4	406	val_406
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42
index 747b650237b5379c16a2704d73c082578edf7f9f..1a82dfc1535657d9e15a3ae35d300591daf64e39 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42
@@ -12,4 +12,4 @@
 311	3	311	val_311
 369	3	369	
 401	5	401	val_401
-406	4	406	val_406
\ No newline at end of file
+406	4	406	val_406
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42
index 747b650237b5379c16a2704d73c082578edf7f9f..1a82dfc1535657d9e15a3ae35d300591daf64e39 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42
@@ -12,4 +12,4 @@
 311	3	311	val_311
 369	3	369	
 401	5	401	val_401
-406	4	406	val_406
\ No newline at end of file
+406	4	406	val_406
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42
index 747b650237b5379c16a2704d73c082578edf7f9f..1a82dfc1535657d9e15a3ae35d300591daf64e39 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42
@@ -12,4 +12,4 @@
 311	3	311	val_311
 369	3	369	
 401	5	401	val_401
-406	4	406	val_406
\ No newline at end of file
+406	4	406	val_406
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d b/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f b/sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f
index 7c0af7229d62ae851e7fafb38f62b30279f2691a..ecbe52c536ebea35ee40755c7f782565b78dee06 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f
@@ -21,4 +21,4 @@ val_273	1	NULL	NULL
 val_278	1	NULL	NULL
 val_311	1	NULL	NULL
 val_401	1	NULL	NULL
-val_406	1	NULL	NULL
\ No newline at end of file
+val_406	1	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 b/sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275
index efdd80c9f8b89a53af4bbf48f22573d1ceb511ee..2f62508e3342a7c2ca92ebfaf906eb942d21e353 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275
@@ -10,4 +10,4 @@
 311	val_311	3
 369		3
 401	val_401	5
-406	val_406	4
\ No newline at end of file
+406	val_406	4
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 b/sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275
index efdd80c9f8b89a53af4bbf48f22573d1ceb511ee..2f62508e3342a7c2ca92ebfaf906eb942d21e353 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275
@@ -10,4 +10,4 @@
 311	val_311	3
 369		3
 401	val_401	5
-406	val_406	4
\ No newline at end of file
+406	val_406	4
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f b/sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f
index 7c0af7229d62ae851e7fafb38f62b30279f2691a..ecbe52c536ebea35ee40755c7f782565b78dee06 100644
--- a/sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f
+++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f
@@ -21,4 +21,4 @@ val_273	1	NULL	NULL
 val_278	1	NULL	NULL
 val_311	1	NULL	NULL
 val_401	1	NULL	NULL
-val_406	1	NULL	NULL
\ No newline at end of file
+val_406	1	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-38-638e5300f4c892c2bf27bd91a8f81b64 b/sql/hive/src/test/resources/golden/count-1-c47c4abedf055b4e734232fd2e274d55
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-38-638e5300f4c892c2bf27bd91a8f81b64
rename to sql/hive/src/test/resources/golden/count-1-c47c4abedf055b4e734232fd2e274d55
diff --git a/sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 b/sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6
index 4b020e0595d2c8035d5e0b3a7a4c984e9d2d2ac1..966f27f6c9b9bb1a8449ebfae626e9af309d729c 100644
--- a/sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6
+++ b/sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6
@@ -1,4 +1,4 @@
 NULL	1	1	6
 10	2	2	10
 12	1	2	9
-100	1	1	3
\ No newline at end of file
+100	1	1	3
diff --git a/sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13 b/sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13
index 54199fdb8166dee51e3014fe27778e3b6f759ca2..5eec149bfcc92c1ac5b3de2103f13ffc895f8703 100644
--- a/sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13
+++ b/sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13
@@ -1 +1 @@
-7	7	6	6	6	7	3	3	6	7	4	5	6	6	5	6	4	5	5	5	4
\ No newline at end of file
+7	7	6	6	6	7	3	3	6	7	4	5	6	6	5	6	4	5	5	5	4
diff --git a/sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 b/sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698
index 2e9278da888588128c23e649ad128482b3f35c65..162877fdb3e5d0f4f0bb870b59e7ce2849982aff 100644
--- a/sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698
+++ b/sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698
@@ -4,4 +4,4 @@ NULL	35	23	6
 12	NULL	80	2
 10	100	NULL	5
 10	100	45	4
-12	100	75	7
\ No newline at end of file
+12	100	75	7
diff --git a/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 b/sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6
index 4b020e0595d2c8035d5e0b3a7a4c984e9d2d2ac1..966f27f6c9b9bb1a8449ebfae626e9af309d729c 100644
--- a/sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6
+++ b/sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6
@@ -1,4 +1,4 @@
 NULL	1	1	6
 10	2	2	10
 12	1	2	9
-100	1	1	3
\ No newline at end of file
+100	1	1	3
diff --git a/sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13 b/sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13
index 54199fdb8166dee51e3014fe27778e3b6f759ca2..5eec149bfcc92c1ac5b3de2103f13ffc895f8703 100644
--- a/sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13
+++ b/sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13
@@ -1 +1 @@
-7	7	6	6	6	7	3	3	6	7	4	5	6	6	5	6	4	5	5	5	4
\ No newline at end of file
+7	7	6	6	6	7	3	3	6	7	4	5	6	6	5	6	4	5	5	5	4
diff --git a/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f b/sql/hive/src/test/resources/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f
deleted file mode 100644
index 7aae61e5eb82f6ed0a954ef086bae93dca6546e8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f	
+++ /dev/null
@@ -1,500 +0,0 @@
-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/create table as-1-b9002c1d71895be765575b62656d1928 b/sql/hive/src/test/resources/golden/create table as-1-b9002c1d71895be765575b62656d1928
index c5c8d29fdd13e6dee2330bc99daedf917717dbbb..7aae61e5eb82f6ed0a954ef086bae93dca6546e8 100644
--- a/sql/hive/src/test/resources/golden/create table as-1-b9002c1d71895be765575b62656d1928	
+++ b/sql/hive/src/test/resources/golden/create table as-1-b9002c1d71895be765575b62656d1928	
@@ -497,4 +497,4 @@
 403	val_403
 400	val_400
 200	val_200
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d b/sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba b/sql/hive/src/test/resources/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba
deleted file mode 100644
index 66d40e52a453978afc551cfe8891badb174c5e08..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba
+++ /dev/null
@@ -1,4 +0,0 @@
-a                   	string              	None                
-b                   	string              	None                
-	 	 
-Detailed Table Information	Table(tableName:table3, dbName:default, owner:marmbrus, createTime:1389343868, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table3, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=	, field.delim=
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 b/sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785
deleted file mode 100644
index 8e606f61a1c30ae1f207b5dea177526704f4291d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785
+++ /dev/null
@@ -1,2 +0,0 @@
-a                   	string              	None                
-b                   	string              	None                
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a b/sql/hive/src/test/resources/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a
deleted file mode 100644
index 05b460270525dbd4f50cb85542ee4df344ac08ed..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a
+++ /dev/null
@@ -1,4 +0,0 @@
-a                   	string              	None                
-b                   	string              	None                
-	 	 
-Detailed Table Information	Table(tableName:table4, dbName:default, owner:marmbrus, createTime:1389343869, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table4, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=	, field.delim=
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 b/sql/hive/src/test/resources/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816
deleted file mode 100644
index eda3946588e3f2176baeba957d29ef32d33ad1a6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816
+++ /dev/null
@@ -1,2 +0,0 @@
-a                   	string              	from deserializer   
-b                   	string              	from deserializer   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 b/sql/hive/src/test/resources/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32
deleted file mode 100644
index 8fc60adf10167ba7381a5a29ad8c7a51a4a882f7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32
+++ /dev/null
@@ -1,4 +0,0 @@
-a                   	string              	from deserializer   
-b                   	string              	from deserializer   
-	 	 
-Detailed Table Information	Table(tableName:table5, dbName:default, owner:marmbrus, createTime:1389343869, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table5, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=	, field.delim=
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084
deleted file mode 100644
index 8e606f61a1c30ae1f207b5dea177526704f4291d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084
+++ /dev/null
@@ -1,2 +0,0 @@
-a                   	string              	None                
-b                   	string              	None                
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 b/sql/hive/src/test/resources/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365
deleted file mode 100644
index 5e9bc70c3a1a30218d8c451b6e8f724753e31266..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365
+++ /dev/null
@@ -1,4 +0,0 @@
-a                   	string              	None                
-b                   	string              	None                
-	 	 
-Detailed Table Information	Table(tableName:table1, dbName:default, owner:marmbrus, createTime:1389343868, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389343868}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a b/sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a
deleted file mode 100644
index 6ed3515ba687657ab789d815b43e56dfe7336bc4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a
+++ /dev/null
@@ -1,2 +0,0 @@
-a                   	string              	None                
-b                   	int                 	None                
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 b/sql/hive/src/test/resources/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001
deleted file mode 100644
index bec4c72e49857be834135d666a0ef95099129529..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001
+++ /dev/null
@@ -1,4 +0,0 @@
-a                   	string              	None                
-b                   	int                 	None                
-	 	 
-Detailed Table Information	Table(tableName:table2, dbName:default, owner:marmbrus, createTime:1389343868, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:int, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389343868}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 b/sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3
deleted file mode 100644
index 8e606f61a1c30ae1f207b5dea177526704f4291d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3
+++ /dev/null
@@ -1,2 +0,0 @@
-a                   	string              	None                
-b                   	string              	None                
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084
deleted file mode 100644
index 8e606f61a1c30ae1f207b5dea177526704f4291d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084
+++ /dev/null
@@ -1,2 +0,0 @@
-a                   	string              	None                
-b                   	string              	None                
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 b/sql/hive/src/test/resources/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365
deleted file mode 100644
index 8ab6b24deab600e8d710a8b5a664bb481dc861fa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365
+++ /dev/null
@@ -1,4 +0,0 @@
-a                   	string              	None                
-b                   	string              	None                
-	 	 
-Detailed Table Information	Table(tableName:table1, dbName:default, owner:marmbrus, createTime:1388825524, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3286459698772672096/table1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{escape.delim=\, serialization.format=	, field.delim=
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 b/sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733
deleted file mode 100644
index bbe37f8e2a790434f1100dc8de1a96cf6b8d0558..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733
+++ /dev/null
@@ -1 +0,0 @@
-86	\	\
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 b/sql/hive/src/test/resources/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717
deleted file mode 100644
index cbcdfbe72e8c6cf58ae9ffdcc8d2649a6a149236..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717
+++ /dev/null
@@ -1,9 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_CREATEFUNCTION test_avg 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage')
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-
diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 b/sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8
deleted file mode 100644
index 2f958c483a9df06e37a4c4f66725a6db4021127f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8
+++ /dev/null
@@ -1 +0,0 @@
-1.0	260.182
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 b/sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1
index 0e9c06c882602b0d9e89694852d9aafd82b74af8..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 100644
--- a/sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1
+++ b/sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1
@@ -1,10 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_CREATEFUNCTION test_translate 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestTranslate')
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-
-
diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc b/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc
index 83f572c424926bcf22eb289e6d8a939d4a67425b..90f9bd0430a4cafabf687a63ca77f97f946c1b22 100644
--- a/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc
+++ b/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc
@@ -1,2 +1,2 @@
-key                 	int                 	None                
-value               	string              	None                
\ No newline at end of file
+key                 	int                 	                    
+value               	string              	                    
diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c b/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c
index 83f572c424926bcf22eb289e6d8a939d4a67425b..90f9bd0430a4cafabf687a63ca77f97f946c1b22 100644
--- a/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c
+++ b/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c
@@ -1,2 +1,2 @@
-key                 	int                 	None                
-value               	string              	None                
\ No newline at end of file
+key                 	int                 	                    
+value               	string              	                    
diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 b/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5
index 83f572c424926bcf22eb289e6d8a939d4a67425b..90f9bd0430a4cafabf687a63ca77f97f946c1b22 100644
--- a/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5
+++ b/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5
@@ -1,2 +1,2 @@
-key                 	int                 	None                
-value               	string              	None                
\ No newline at end of file
+key                 	int                 	                    
+value               	string              	                    
diff --git a/sql/hive/src/test/resources/golden/create_like-1-30485a2507b60b96cad3d293527e6af b/sql/hive/src/test/resources/golden/create_like-1-30485a2507b60b96cad3d293527e6af
deleted file mode 100644
index b8ddba7f50b972cb3b32cfa8e48d90a12c64d043..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_like-1-30485a2507b60b96cad3d293527e6af
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-a                   	string              	None                
-b                   	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 11:13:23 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/table1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392059603          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 b/sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733
deleted file mode 100644
index e87fb81a6f043a3b0c9760c6afc59c3f8d7b7d7d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733
+++ /dev/null
@@ -1 +0,0 @@
-86	val_86
diff --git a/sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130 b/sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130
deleted file mode 100644
index 375dcacb8db713055c0e6786f40a1051b8c3a2dd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130
+++ /dev/null
@@ -1,2 +0,0 @@
-100	val_100
-100	val_100
diff --git a/sql/hive/src/test/resources/golden/create_like-3-eea111a209cf8895f31f64699669c705 b/sql/hive/src/test/resources/golden/create_like-3-eea111a209cf8895f31f64699669c705
deleted file mode 100644
index 52b637c27f98deb9b6c631a9a3abf60558f42510..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_like-3-eea111a209cf8895f31f64699669c705
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-a                   	string              	None                
-b                   	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 11:13:23 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/table2	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392059603          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 b/sql/hive/src/test/resources/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7
deleted file mode 100644
index d9308798dcb1c3f4c37cf63cc86e2461e90da7e5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7
+++ /dev/null
@@ -1,28 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-a                   	string              	None                
-b                   	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 11:13:23 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/table3	 
-Table Type:         	EXTERNAL_TABLE      	 
-Table Parameters:	 	 
-	EXTERNAL            	TRUE                
-	transient_lastDdlTime	1392059603          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 b/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705
index 3e2a912824ab3400f626c68b9da2a9395e03477a..0e52b5f82ef4a8fa9fea76bef813a68a434b0f66 100644
--- a/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705
+++ b/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705
@@ -1,19 +1,19 @@
 # col_name            	data_type           	comment             
 	 	 
-a                   	string              	None                
-b                   	string              	None                
+a                   	string              	                    
+b                   	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Fri Feb 07 14:52:37 PST 2014	 
+CreateTime:         	Tue Oct 21 01:24:36 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table2	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table2	 
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1391813557          
+	transient_lastDdlTime	1413879876          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
diff --git a/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 b/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3
index 115d12fb70c818b01166a3acf42da97b9ab25dac..26b59894887523839d5200c55ebf63a39772eada 100644
--- a/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3
+++ b/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3
@@ -1,2 +1,2 @@
-a                   	string              	None                
-b                   	string              	None                
+a                   	string              	                    
+b                   	string              	                    
diff --git a/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 b/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7
index 61d34badb1a2d2f07d3dc4fdcf9ca0fe69eb9924..47808e8f20c839808d7960f12ad7a73b7a6622e1 100644
--- a/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7
+++ b/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7
@@ -1,20 +1,20 @@
 # col_name            	data_type           	comment             
 	 	 
-a                   	string              	None                
-b                   	string              	None                
+a                   	string              	                    
+b                   	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Fri Feb 07 14:52:37 PST 2014	 
+CreateTime:         	Tue Oct 21 01:24:36 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table3	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table3	 
 Table Type:         	EXTERNAL_TABLE      	 
 Table Parameters:	 	 
 	EXTERNAL            	TRUE                
-	transient_lastDdlTime	1391813557          
+	transient_lastDdlTime	1413879876          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
diff --git a/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d b/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d
index 4c2967215fe66ca8b2464022d6296052b0c9ead1..ad1f1f42d6b98b68f0ebfecd82c06637abf057a1 100644
--- a/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d
+++ b/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d
@@ -1,25 +1,25 @@
 # col_name            	data_type           	comment             
 	 	 
-key                 	int                 	None                
-value               	string              	None                
+key                 	int                 	                    
+value               	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-hr                  	string              	None                
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Fri Feb 07 14:53:16 PST 2014	 
+CreateTime:         	Tue Oct 21 01:25:10 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table1	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table1	 
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1391813596          
+	transient_lastDdlTime	1413879910          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
diff --git a/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084
index 115d12fb70c818b01166a3acf42da97b9ab25dac..26b59894887523839d5200c55ebf63a39772eada 100644
--- a/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084
+++ b/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084
@@ -1,2 +1,2 @@
-a                   	string              	None                
-b                   	string              	None                
+a                   	string              	                    
+b                   	string              	                    
diff --git a/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af b/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af
index 02dee147bca429f9cde7d244e5a5aed125be03aa..91e1ebbfee4ded40b656ae1f67e48885fd36c566 100644
--- a/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af
+++ b/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af
@@ -1,19 +1,19 @@
 # col_name            	data_type           	comment             
 	 	 
-a                   	string              	None                
-b                   	string              	None                
+a                   	string              	                    
+b                   	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Fri Feb 07 14:52:37 PST 2014	 
+CreateTime:         	Tue Oct 21 01:24:36 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table1	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table1	 
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1391813557          
+	transient_lastDdlTime	1413879876          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
diff --git a/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a b/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a
index 115d12fb70c818b01166a3acf42da97b9ab25dac..26b59894887523839d5200c55ebf63a39772eada 100644
--- a/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a
+++ b/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a
@@ -1,2 +1,2 @@
-a                   	string              	None                
-b                   	string              	None                
+a                   	string              	                    
+b                   	string              	                    
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 b/sql/hive/src/test/resources/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929
deleted file mode 100644
index dbdf4585360ae8140797b398a691efe2d68ea4b7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:tgt_rc_merge_test
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/tgt_rc_merge_test
-inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat
-columns:struct columns { i32 key, string value}
-partitioned:false
-partitionColumns:
-totalNumberFiles:1
-totalFileSize:239
-maxFileSize:239
-minFileSize:239
-lastAccessTime:0
-lastUpdateTime:1389344017000
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d b/sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d
deleted file mode 100644
index 9a037142aa3c1b4c490e1a38251620f113465330..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d
+++ /dev/null
@@ -1 +0,0 @@
-10
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 b/sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320
deleted file mode 100644
index eb6c3f6aef81353d4f60a0bc8aa71c88abbabbac..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320
+++ /dev/null
@@ -1 +0,0 @@
-508	-751895388
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 b/sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 b/sql/hive/src/test/resources/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929
deleted file mode 100644
index 8a9d9d56a66d6765d97e811005437b31d671f946..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:tgt_rc_merge_test
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/tgt_rc_merge_test
-inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat
-columns:struct columns { i32 key, string value}
-partitioned:false
-partitionColumns:
-totalNumberFiles:2
-totalFileSize:338
-maxFileSize:169
-minFileSize:169
-lastAccessTime:0
-lastUpdateTime:1389343990000
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d b/sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d
deleted file mode 100644
index 9a037142aa3c1b4c490e1a38251620f113465330..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d
+++ /dev/null
@@ -1 +0,0 @@
-10
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 b/sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320
deleted file mode 100644
index eb6c3f6aef81353d4f60a0bc8aa71c88abbabbac..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320
+++ /dev/null
@@ -1 +0,0 @@
-508	-751895388
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084
index c49ad26c04d67899bd5dd909ec098b36c806cd2d..6485e594f54fee95cdad26399a106cbccec93701 100644
--- a/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084
+++ b/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084
@@ -1,4 +1,4 @@
-a                   	string              	None                
-b                   	array<string>       	None                
-c                   	array<map<string,string>>	None                
-d                   	map<string,array<string>>	None                
+a                   	string              	                    
+b                   	array<string>       	                    
+c                   	array<map<string,string>>	                    
+d                   	map<string,array<string>>	                    
diff --git a/sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 b/sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365
index 41c7202c8b88643ca037dc5794f198939ad20e27..6d68db6387495484a7e3697851fb8669964268f6 100644
--- a/sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365
+++ b/sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365
@@ -1,6 +1,6 @@
-a                   	string              	None                
-b                   	array<string>       	None                
-c                   	array<map<string,string>>	None                
-d                   	map<string,array<string>>	None                
+a                   	string              	                    
+b                   	array<string>       	                    
+c                   	array<map<string,string>>	                    
+d                   	map<string,array<string>>	                    
 	 	 
-Detailed Table Information	Table(tableName:table1, dbName:default, owner:marmbrus, createTime:1391226109, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:array<string>, comment:null), FieldSchema(name:c, type:array<map<string,string>>, comment:null), FieldSchema(name:d, type:map<string,array<string>>, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/table1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1391226109}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:table1, dbName:default, owner:marmbrus, createTime:1413879912, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:array<string>, comment:null), FieldSchema(name:c, type:array<map<string,string>>, comment:null), FieldSchema(name:d, type:map<string,array<string>>, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413879912}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-41-3514c74c7f68f2d70cc6d51ac46c20 b/sql/hive/src/test/resources/golden/create_nested_type-3-280fd0d9876e475d7dcf5383876ebc79
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-41-3514c74c7f68f2d70cc6d51ac46c20
rename to sql/hive/src/test/resources/golden/create_nested_type-3-280fd0d9876e475d7dcf5383876ebc79
diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 b/sql/hive/src/test/resources/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13
deleted file mode 100644
index 46869fc9e88ebf71ee11fdac9cef2b961da0a27f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13
+++ /dev/null
@@ -1,30 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 12:09:28 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Table Type:         	VIRTUAL_VIEW        	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392062968          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	null                	 
-InputFormat:        	org.apache.hadoop.mapred.SequenceFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-	 	 
-# View Information	 	 
-View Original Text: 	select * from srcpart	 
-View Expanded Text: 	select `srcpart`.`key`, `srcpart`.`value`, `srcpart`.`ds`, `srcpart`.`hr` from `default`.`srcpart`	 
diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557
+++ b/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 b/sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20
index d7c386e17c096ab9dd6a775d03eff3c56cd644da..d35fc35a810d38dfcf03981b37c7d74c6e8d88ee 100644
--- a/sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20
+++ b/sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20
@@ -1,19 +1,19 @@
 # col_name            	data_type           	comment             
 	 	 
-key                 	string              	None                
-value               	string              	None                
+key                 	string              	                    
+value               	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 14 22:51:44 PST 2014	 
+CreateTime:         	Tue Oct 21 01:25:13 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6750581243740872392/list_bucket_single_2	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/list_bucket_single_2	 
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1389768704          
+	transient_lastDdlTime	1413879913          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
@@ -26,4 +26,4 @@ Sort Columns:       	[]
 Skewed Columns:     	[key]               	 
 Skewed Values:      	[[1], [5], [6]]     	 
 Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e b/sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e
index 0817efac83d79b1869b041cba2b806f03371400f..e8f72845c1c7ae1e29e2b408498d5f8ed5863dfc 100644
--- a/sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e
+++ b/sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e
@@ -1,19 +1,19 @@
 # col_name            	data_type           	comment             
 	 	 
-key                 	string              	None                
-value               	string              	None                
+key                 	string              	                    
+value               	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 14 22:51:44 PST 2014	 
+CreateTime:         	Tue Oct 21 01:25:13 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6750581243740872392/list_bucket_single	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/list_bucket_single	 
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1389768704          
+	transient_lastDdlTime	1413879913          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
@@ -26,4 +26,4 @@ Sort Columns:       	[]
 Skewed Columns:     	[key]               	 
 Skewed Values:      	[[1], [5], [6]]     	 
 Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef b/sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef
index 2986dd43b0fd60ff6293ba6ed0029aaa04acb046..c8d58bbb1b1ce6e949f2b37f824a652ea98e482f 100644
--- a/sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef
+++ b/sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef
@@ -1,20 +1,20 @@
 # col_name            	data_type           	comment             
 	 	 
-col1                	string              	None                
-col2                	int                 	None                
-col3                	string              	None                
+col1                	string              	                    
+col2                	int                 	                    
+col3                	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 14 22:51:44 PST 2014	 
+CreateTime:         	Tue Oct 21 01:25:13 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6750581243740872392/list_bucket_multiple	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/list_bucket_multiple	 
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1389768704          
+	transient_lastDdlTime	1413879913          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
@@ -27,4 +27,4 @@ Sort Columns:       	[]
 Skewed Columns:     	[col1, col2]        	 
 Skewed Values:      	[[s1, 1], [s3, 3], [s13, 13], [s78, 78]]	 
 Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-44-7490df6719cd7e47aa08dbcbc3266a92 b/sql/hive/src/test/resources/golden/create_struct_table-1-719851d0e8b89b51bdc6be4177455a92
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-44-7490df6719cd7e47aa08dbcbc3266a92
rename to sql/hive/src/test/resources/golden/create_struct_table-1-719851d0e8b89b51bdc6be4177455a92
diff --git a/sql/hive/src/test/resources/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e b/sql/hive/src/test/resources/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e
deleted file mode 100644
index 8af6a0338d65a883456d133ac570c6c2c344b85f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e
+++ /dev/null
@@ -1,10 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_CREATEFUNCTION test_max 'org.apache.hadoop.hive.ql.udf.UDAFTestMax')
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-
-
diff --git a/sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b b/sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b
deleted file mode 100644
index e87fb81a6f043a3b0c9760c6afc59c3f8d7b7d7d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b
+++ /dev/null
@@ -1 +0,0 @@
-86	val_86
diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 b/sql/hive/src/test/resources/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5
deleted file mode 100644
index 70d2ecdbc8f78446750711848422d3b1364b5e29..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5
+++ /dev/null
@@ -1,13 +0,0 @@
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-value               	string              	None                
-	 	 
-Detailed Table Information	Table(tableName:vp1, dbName:default, owner:marmbrus, createTime:1392062982, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:null, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:null, parameters:{}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:value, type:string, comment:null)], parameters:{transient_lastDdlTime=1392062982}, viewOriginalText:SELECT key, value	 
-FROM src	 	 
-WHERE key=86, viewExpandedText:SELECT `src`.`key`, `src`.`value`	 	 
-FROM `default`.`src`	 	 
-WHERE `src`.`key`=86, tableType:VIRTUAL_VIEW)		 
diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc b/sql/hive/src/test/resources/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc
deleted file mode 100644
index 43bc3de44f4dfba6578107c9f09e89c2f6080756..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-value               	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 12:09:42 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Table Type:         	VIRTUAL_VIEW        	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392062982          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	null                	 
-InputFormat:        	org.apache.hadoop.mapred.SequenceFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-	 	 
-# View Information	 	 
-View Original Text: 	SELECT key, value	 
-FROM src	 	 
-WHERE key=86	 	 
-View Expanded Text: 	SELECT `src`.`key`, `src`.`value`	 
-FROM `default`.`src`	 	 
-WHERE `src`.`key`=86	 	 
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-47-73da9fe2b0c2ee26c021ec3f2fa27272 b/sql/hive/src/test/resources/golden/cross_product_check_1-0-d782db598869f9b19e0fcf5ea2a83594
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-47-73da9fe2b0c2ee26c021ec3f2fa27272
rename to sql/hive/src/test/resources/golden/cross_product_check_1-0-d782db598869f9b19e0fcf5ea2a83594
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-51-fcf9bcb522f542637ccdea863b408448 b/sql/hive/src/test/resources/golden/cross_product_check_1-1-bd8395c55fa2fc80b68eb043b7020b76
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-51-fcf9bcb522f542637ccdea863b408448
rename to sql/hive/src/test/resources/golden/cross_product_check_1-1-bd8395c55fa2fc80b68eb043b7020b76
diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/cross_product_check_1-2-4e1b3108b7e1b9d8e94e73f9dfa44617
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/cross_product_check_1-2-4e1b3108b7e1b9d8e94e73f9dfa44617
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-54-dad56e1f06c808b29e5dc8fb0c49efb2 b/sql/hive/src/test/resources/golden/cross_product_check_1-3-32a3e6eb858d37f58e225d07fb323254
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-54-dad56e1f06c808b29e5dc8fb0c49efb2
rename to sql/hive/src/test/resources/golden/cross_product_check_1-3-32a3e6eb858d37f58e225d07fb323254
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-57-3cd3fbbbd8ee5c274fe3d6a45126cef4 b/sql/hive/src/test/resources/golden/cross_product_check_1-4-36a6b6fb71ea08ff817dd40d1ffdb970
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-57-3cd3fbbbd8ee5c274fe3d6a45126cef4
rename to sql/hive/src/test/resources/golden/cross_product_check_1-4-36a6b6fb71ea08ff817dd40d1ffdb970
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-60-d6bbaf0d40010159095e4cac025c50c5 b/sql/hive/src/test/resources/golden/cross_product_check_1-5-103739f072b849d212dbc40919f92b74
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer1-60-d6bbaf0d40010159095e4cac025c50c5
rename to sql/hive/src/test/resources/golden/cross_product_check_1-5-103739f072b849d212dbc40919f92b74
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/cross_product_check_1-6-9a202e3bf15be2a310504a50920e7d25
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b
rename to sql/hive/src/test/resources/golden/cross_product_check_1-6-9a202e3bf15be2a310504a50920e7d25
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/cross_product_check_1-7-e4e93f6e0dc63e1e324fb913a26fa8ac
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979
rename to sql/hive/src/test/resources/golden/cross_product_check_1-7-e4e93f6e0dc63e1e324fb913a26fa8ac
diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 b/sql/hive/src/test/resources/golden/cross_product_check_2-0-d782db598869f9b19e0fcf5ea2a83594
similarity index 100%
rename from sql/hive/src/test/resources/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9
rename to sql/hive/src/test/resources/golden/cross_product_check_2-0-d782db598869f9b19e0fcf5ea2a83594
diff --git a/sql/hive/src/test/resources/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 b/sql/hive/src/test/resources/golden/cross_product_check_2-1-bd8395c55fa2fc80b68eb043b7020b76
similarity index 100%
rename from sql/hive/src/test/resources/golden/count-1-3531872d964bc2b4f07d51b9d1629df2
rename to sql/hive/src/test/resources/golden/cross_product_check_2-1-bd8395c55fa2fc80b68eb043b7020b76
diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/cross_product_check_2-2-24ca942f094b14b92086305cc125e833
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/cross_product_check_2-2-24ca942f094b14b92086305cc125e833
diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/cross_product_check_2-3-44d382ce6848d3f0b900b0808747d8e9
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/cross_product_check_2-3-44d382ce6848d3f0b900b0808747d8e9
diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/cross_product_check_2-4-c14792ccac2ca64e3e9e21af4fd12d2c
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/cross_product_check_2-4-c14792ccac2ca64e3e9e21af4fd12d2c
diff --git a/sql/hive/src/test/resources/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 b/sql/hive/src/test/resources/golden/cross_product_check_2-5-32a3e6eb858d37f58e225d07fb323254
similarity index 100%
rename from sql/hive/src/test/resources/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425
rename to sql/hive/src/test/resources/golden/cross_product_check_2-5-32a3e6eb858d37f58e225d07fb323254
diff --git a/sql/hive/src/test/resources/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 b/sql/hive/src/test/resources/golden/cross_product_check_2-6-36a6b6fb71ea08ff817dd40d1ffdb970
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3
rename to sql/hive/src/test/resources/golden/cross_product_check_2-6-36a6b6fb71ea08ff817dd40d1ffdb970
diff --git a/sql/hive/src/test/resources/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 b/sql/hive/src/test/resources/golden/cross_product_check_2-7-103739f072b849d212dbc40919f92b74
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5
rename to sql/hive/src/test/resources/golden/cross_product_check_2-7-103739f072b849d212dbc40919f92b74
diff --git a/sql/hive/src/test/resources/golden/create_1-14-437bd1acbae61f48e851addb769d3379 b/sql/hive/src/test/resources/golden/cross_product_check_2-8-9a202e3bf15be2a310504a50920e7d25
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_1-14-437bd1acbae61f48e851addb769d3379
rename to sql/hive/src/test/resources/golden/cross_product_check_2-8-9a202e3bf15be2a310504a50920e7d25
diff --git a/sql/hive/src/test/resources/golden/create_1-4-610b82bf7b0080d293977927e5ef780c b/sql/hive/src/test/resources/golden/cross_product_check_2-9-e4e93f6e0dc63e1e324fb913a26fa8ac
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_1-4-610b82bf7b0080d293977927e5ef780c
rename to sql/hive/src/test/resources/golden/cross_product_check_2-9-e4e93f6e0dc63e1e324fb913a26fa8ac
diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec b/sql/hive/src/test/resources/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec
deleted file mode 100644
index 6839c16243bcdceb7d6f6e847e095cc915514fbc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec
+++ /dev/null
@@ -1,5 +0,0 @@
-0	val_0
-0	val_0
-0	val_0
-2	val_2
-4	val_4
diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 b/sql/hive/src/test/resources/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32
deleted file mode 100644
index 6839c16243bcdceb7d6f6e847e095cc915514fbc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32
+++ /dev/null
@@ -1,5 +0,0 @@
-0	val_0
-0	val_0
-0	val_0
-2	val_2
-4	val_4
diff --git a/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd b/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd	
+++ b/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd	
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e b/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e
+++ b/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/create_1-5-c77b018276b1558c1d9462e0625e152e b/sql/hive/src/test/resources/golden/database_drop-10-8db536f925bf0f5058f97897e145a661
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_1-5-c77b018276b1558c1d9462e0625e152e
rename to sql/hive/src/test/resources/golden/database_drop-10-8db536f925bf0f5058f97897e145a661
diff --git a/sql/hive/src/test/resources/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 b/sql/hive/src/test/resources/golden/database_drop-11-1b0a6cff3151cfa0ef0a6f78587973a5
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59
rename to sql/hive/src/test/resources/golden/database_drop-11-1b0a6cff3151cfa0ef0a6f78587973a5
diff --git a/sql/hive/src/test/resources/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 b/sql/hive/src/test/resources/golden/database_drop-12-2ea883422b74b701711e14e61472ba06
similarity index 100%
rename from sql/hive/src/test/resources/golden/database_drop-11-2ea883422b74b701711e14e61472ba06
rename to sql/hive/src/test/resources/golden/database_drop-12-2ea883422b74b701711e14e61472ba06
diff --git a/sql/hive/src/test/resources/golden/database_drop-12-e02a53f7e798d2741152526516f14941 b/sql/hive/src/test/resources/golden/database_drop-13-e02a53f7e798d2741152526516f14941
similarity index 100%
rename from sql/hive/src/test/resources/golden/database_drop-12-e02a53f7e798d2741152526516f14941
rename to sql/hive/src/test/resources/golden/database_drop-13-e02a53f7e798d2741152526516f14941
diff --git a/sql/hive/src/test/resources/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d b/sql/hive/src/test/resources/golden/database_drop-14-8f0fe60664d020b2a42c7f5c0c7bed35
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d
rename to sql/hive/src/test/resources/golden/database_drop-14-8f0fe60664d020b2a42c7f5c0c7bed35
diff --git a/sql/hive/src/test/resources/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee b/sql/hive/src/test/resources/golden/database_drop-15-7928ac876f76c990fa21d74b6c9e14f6
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee
rename to sql/hive/src/test/resources/golden/database_drop-15-7928ac876f76c990fa21d74b6c9e14f6
diff --git a/sql/hive/src/test/resources/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e b/sql/hive/src/test/resources/golden/database_drop-16-f9036ff3f0a3101373bdbc9a52faf00e
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e
rename to sql/hive/src/test/resources/golden/database_drop-16-f9036ff3f0a3101373bdbc9a52faf00e
diff --git a/sql/hive/src/test/resources/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 b/sql/hive/src/test/resources/golden/database_drop-17-95b906fd73935da8746b5277170e91e8
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14
rename to sql/hive/src/test/resources/golden/database_drop-17-95b906fd73935da8746b5277170e91e8
diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b b/sql/hive/src/test/resources/golden/database_drop-18-f65bf675b83871af7906741a60fa1318
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b
rename to sql/hive/src/test/resources/golden/database_drop-18-f65bf675b83871af7906741a60fa1318
diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 b/sql/hive/src/test/resources/golden/database_drop-19-d7fefbf585dbb67491e871ef58dca752
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2
rename to sql/hive/src/test/resources/golden/database_drop-19-d7fefbf585dbb67491e871ef58dca752
diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 b/sql/hive/src/test/resources/golden/database_drop-20-cbb84f0db4d55930a85cff28f7400bd0
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1
rename to sql/hive/src/test/resources/golden/database_drop-20-cbb84f0db4d55930a85cff28f7400bd0
diff --git a/sql/hive/src/test/resources/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 b/sql/hive/src/test/resources/golden/database_drop-21-eea49f066631be60519ae9d6b614d7d0
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3
rename to sql/hive/src/test/resources/golden/database_drop-21-eea49f066631be60519ae9d6b614d7d0
diff --git a/sql/hive/src/test/resources/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a b/sql/hive/src/test/resources/golden/database_drop-22-85833c3a68c29098827e438ff580bb94
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a
rename to sql/hive/src/test/resources/golden/database_drop-22-85833c3a68c29098827e438ff580bb94
diff --git a/sql/hive/src/test/resources/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 b/sql/hive/src/test/resources/golden/database_drop-23-84a5672989118a1b5792474c1469de90
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86
rename to sql/hive/src/test/resources/golden/database_drop-23-84a5672989118a1b5792474c1469de90
diff --git a/sql/hive/src/test/resources/golden/create_like-5-dc9de26002604e9e436135bd4b40636d b/sql/hive/src/test/resources/golden/database_drop-24-441116797e8d95554c74472fa7644440
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_like-5-dc9de26002604e9e436135bd4b40636d
rename to sql/hive/src/test/resources/golden/database_drop-24-441116797e8d95554c74472fa7644440
diff --git a/sql/hive/src/test/resources/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 b/sql/hive/src/test/resources/golden/database_drop-25-847ca97dd211889d83e5d487bfc44e23
similarity index 100%
rename from sql/hive/src/test/resources/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0
rename to sql/hive/src/test/resources/golden/database_drop-25-847ca97dd211889d83e5d487bfc44e23
diff --git a/sql/hive/src/test/resources/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 b/sql/hive/src/test/resources/golden/database_drop-26-bea20178515df24fcca04c0384f1c1b7
similarity index 100%
rename from sql/hive/src/test/resources/golden/database_drop-4-d419f4ff197d4291208c2028cd158909
rename to sql/hive/src/test/resources/golden/database_drop-26-bea20178515df24fcca04c0384f1c1b7
diff --git a/sql/hive/src/test/resources/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 b/sql/hive/src/test/resources/golden/database_drop-27-cb4b90a7f63c1646319ee7bb014a7750
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919
rename to sql/hive/src/test/resources/golden/database_drop-27-cb4b90a7f63c1646319ee7bb014a7750
diff --git a/sql/hive/src/test/resources/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb b/sql/hive/src/test/resources/golden/database_drop-28-53d67cbed634cff012dac41340bf6630
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb
rename to sql/hive/src/test/resources/golden/database_drop-28-53d67cbed634cff012dac41340bf6630
diff --git a/sql/hive/src/test/resources/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb b/sql/hive/src/test/resources/golden/database_drop-29-29d3232325eda40cbb0bd1786b7d070e
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb
rename to sql/hive/src/test/resources/golden/database_drop-29-29d3232325eda40cbb0bd1786b7d070e
diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/database_drop-3-db64b724719d27c7f0db4f51f5c4edaa
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/database_drop-3-db64b724719d27c7f0db4f51f5c4edaa
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c b/sql/hive/src/test/resources/golden/database_drop-30-bbad0860316d8b9b1eed50d231f3ab5d
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c
rename to sql/hive/src/test/resources/golden/database_drop-30-bbad0860316d8b9b1eed50d231f3ab5d
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 b/sql/hive/src/test/resources/golden/database_drop-31-981f8e58caeae9cbbad3a113e3043de5
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342
rename to sql/hive/src/test/resources/golden/database_drop-31-981f8e58caeae9cbbad3a113e3043de5
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f b/sql/hive/src/test/resources/golden/database_drop-32-6c8e7590238b5aca1772721f0b914ece
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f
rename to sql/hive/src/test/resources/golden/database_drop-32-6c8e7590238b5aca1772721f0b914ece
diff --git a/sql/hive/src/test/resources/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 b/sql/hive/src/test/resources/golden/database_drop-33-2bc7864932f597bdf98bdc410b1a2d9c
similarity index 100%
rename from sql/hive/src/test/resources/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24
rename to sql/hive/src/test/resources/golden/database_drop-33-2bc7864932f597bdf98bdc410b1a2d9c
diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/database_drop-34-491138bed44a70cb783bb2b531e1d82
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32
rename to sql/hive/src/test/resources/golden/database_drop-34-491138bed44a70cb783bb2b531e1d82
diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/database_drop-35-9e0285d0596607cdadf75a763a543866
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5
rename to sql/hive/src/test/resources/golden/database_drop-35-9e0285d0596607cdadf75a763a543866
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 b/sql/hive/src/test/resources/golden/database_drop-36-e66471f3f1bbe2d4b3b214e47793656d
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5
rename to sql/hive/src/test/resources/golden/database_drop-36-e66471f3f1bbe2d4b3b214e47793656d
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 b/sql/hive/src/test/resources/golden/database_drop-37-f6410721e3125a89836817136306eac4
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3
rename to sql/hive/src/test/resources/golden/database_drop-37-f6410721e3125a89836817136306eac4
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 b/sql/hive/src/test/resources/golden/database_drop-38-7d45d97adebe50917a94cbe232c112a8
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3
rename to sql/hive/src/test/resources/golden/database_drop-38-7d45d97adebe50917a94cbe232c112a8
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 b/sql/hive/src/test/resources/golden/database_drop-39-91b4a660ae5d5d2966d6bf3b6ae751d1
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3
rename to sql/hive/src/test/resources/golden/database_drop-39-91b4a660ae5d5d2966d6bf3b6ae751d1
diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/database_drop-4-f21dd011aeb417043ed58c03fd5c3bf0
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32
rename to sql/hive/src/test/resources/golden/database_drop-4-f21dd011aeb417043ed58c03fd5c3bf0
diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 b/sql/hive/src/test/resources/golden/database_drop-40-10073fb74a5c792322fc52389997695b
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5
rename to sql/hive/src/test/resources/golden/database_drop-40-10073fb74a5c792322fc52389997695b
diff --git a/sql/hive/src/test/resources/golden/create_nested_type-3-ac452c9279877935983c37113898e53c b/sql/hive/src/test/resources/golden/database_drop-41-7164c585e1ef4d9036ed4db275811084
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_nested_type-3-ac452c9279877935983c37113898e53c
rename to sql/hive/src/test/resources/golden/database_drop-41-7164c585e1ef4d9036ed4db275811084
diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d b/sql/hive/src/test/resources/golden/database_drop-42-c55cffbfc4d950363be97ccdb028faf3
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d
rename to sql/hive/src/test/resources/golden/database_drop-42-c55cffbfc4d950363be97ccdb028faf3
diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/database_drop-43-cc6860c125b8b62450cb858c72716dc2
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5
rename to sql/hive/src/test/resources/golden/database_drop-43-cc6860c125b8b62450cb858c72716dc2
diff --git a/sql/hive/src/test/resources/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/database_drop-44-de81fd80132350aedcd9f0d9a212fd94
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/database_drop-44-de81fd80132350aedcd9f0d9a212fd94
diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea b/sql/hive/src/test/resources/golden/database_drop-45-64e22634663e57153119340583e93651
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea
rename to sql/hive/src/test/resources/golden/database_drop-45-64e22634663e57153119340583e93651
diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d b/sql/hive/src/test/resources/golden/database_drop-46-eb28b907b605e51026f9902287e1d90d
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d
rename to sql/hive/src/test/resources/golden/database_drop-46-eb28b907b605e51026f9902287e1d90d
diff --git a/sql/hive/src/test/resources/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/database_drop-47-44e4d2bdab2dfa4583da47281ed00ba3
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/database_drop-47-44e4d2bdab2dfa4583da47281ed00ba3
diff --git a/sql/hive/src/test/resources/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/database_drop-48-d419f4ff197d4291208c2028cd158909
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/database_drop-48-d419f4ff197d4291208c2028cd158909
diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/database_drop-5-d419f4ff197d4291208c2028cd158909
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22
rename to sql/hive/src/test/resources/golden/database_drop-5-d419f4ff197d4291208c2028cd158909
diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/database_drop-6-b7cf74929eabe781b0db79ed1043dc24
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07
rename to sql/hive/src/test/resources/golden/database_drop-6-b7cf74929eabe781b0db79ed1043dc24
diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de b/sql/hive/src/test/resources/golden/database_drop-7-a47b1b070ec7c3b9ccabc34f41aebad9
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de
rename to sql/hive/src/test/resources/golden/database_drop-7-a47b1b070ec7c3b9ccabc34f41aebad9
diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb b/sql/hive/src/test/resources/golden/database_drop-8-b3980119a4b2d8f531951a940cba3697
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb
rename to sql/hive/src/test/resources/golden/database_drop-8-b3980119a4b2d8f531951a940cba3697
diff --git a/sql/hive/src/test/resources/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 b/sql/hive/src/test/resources/golden/database_drop-9-97101266791d2b2c662bcde549422318
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294
rename to sql/hive/src/test/resources/golden/database_drop-9-97101266791d2b2c662bcde549422318
diff --git a/sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 b/sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491
index f05305df0c3d3f3d13b081cd53177c1030cbfd7d..a97bfdfeef0986a34152afad37f3f2f838fe5f22 100644
--- a/sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491
+++ b/sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491
@@ -1 +1 @@
-db1		file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db	
\ No newline at end of file
+db1		file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db1.db	marmbrus
diff --git a/sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 b/sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45
index e92c241d50092d98eca713c9de077a0a3b807ccf..adf00309e0faed0ddb3f1dc464ef0d1717bf3ce7 100644
--- a/sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45
+++ b/sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45
@@ -1,19 +1,19 @@
 # col_name            	data_type           	comment             
 	 	 
-name                	string              	None                
-value               	int                 	None                
+name                	string              	                    
+value               	int                 	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	db2                 	 
 Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 14 11:40:51 PST 2014	 
+CreateTime:         	Tue Oct 21 01:25:24 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2/table_db2	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2/table_db2	 
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1389728451          
+	transient_lastDdlTime	1413879924          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
@@ -24,4 +24,4 @@ Num Buckets:        	-1
 Bucket Columns:     	[]                  	 
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c
index 7c77c06cda82bc5ce320cfb273eed4e7223ba717..2e4bf94808285485032aae8eed8a6eb6f0712e35 100644
--- a/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c
@@ -1 +1 @@
-table_db2
\ No newline at end of file
+table_db2
diff --git a/sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb b/sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb
index cda1736e9ca6bdd10b01ef955f49f55b1e225943..e7c766a71c1d28cbd480ed3f57472965612b70fc 100644
--- a/sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb
+++ b/sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb
@@ -1,19 +1,19 @@
 # col_name            	data_type           	comment             
 	 	 
-name                	string              	None                
-value               	int                 	None                
+name                	string              	                    
+value               	int                 	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	db1                 	 
 Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 14 11:40:50 PST 2014	 
+CreateTime:         	Tue Oct 21 01:25:24 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/table_db1	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db1.db/table_db1	 
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1389728450          
+	transient_lastDdlTime	1413879924          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
@@ -24,4 +24,4 @@ Num Buckets:        	-1
 Bucket Columns:     	[]                  	 
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c
index 4d8fc6a6d6fe03b8c7bfc382eb66d08667d12489..6eabc06bb13629ad0e5e5a492377721d4e422163 100644
--- a/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c
@@ -1 +1 @@
-table_db1
\ No newline at end of file
+table_db1
diff --git a/sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 b/sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2
index 3e23970adddcf436590c65c447d08d5e8330630a..65f9d79a3c381d820f0984934106d7464b6d79d3 100644
--- a/sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2
+++ b/sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2
@@ -1 +1 @@
-db2	database 2	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2	
\ No newline at end of file
+db2	database 2	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2	marmbrus
diff --git a/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f
+++ b/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 b/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68
+++ b/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 b/sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23
index 2f5fbe26f4945bbc9679da25a4dce05d46eed407..4c04259aed3a7d3ad2784df04a0108834bce6711 100644
--- a/sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23
+++ b/sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23
@@ -1 +1 @@
-db2		file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2.db	{mapred.jobtracker.url=http://my.jobtracker.com:53000, new.property=some new props, mapred.scratch.dir=hdfs://tmp.dfs.com:50029/tmp, hive.warehouse.dir=new/warehouse/dir}
\ No newline at end of file
+db2		file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2.db	marmbrus
diff --git a/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 b/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9
index 86362ead004d9fbe96e8f8e5d1765bd1f33cf35f..3cd776a0711ff5f104cd9ec97f8611a80529879c 100644
--- a/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9
+++ b/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9
@@ -1,2 +1,2 @@
 db1
-default
\ No newline at end of file
+default
diff --git a/sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a b/sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a
index ff89c3fe899d2656eff2644a498a3c57504e8127..4c04259aed3a7d3ad2784df04a0108834bce6711 100644
--- a/sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a
+++ b/sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a
@@ -1 +1 @@
-db2		file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2.db	
\ No newline at end of file
+db2		file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2.db	marmbrus
diff --git a/sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 b/sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23
index 5827d2726d084b32cb751e8eb33986feb8e8ef5d..4c04259aed3a7d3ad2784df04a0108834bce6711 100644
--- a/sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23
+++ b/sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23
@@ -1 +1 @@
-db2		file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2.db	{mapred.jobtracker.url=http://my.jobtracker.com:53000, mapred.scratch.dir=hdfs://tmp.dfs.com:50029/tmp, hive.warehouse.dir=/user/hive/warehouse}
\ No newline at end of file
+db2		file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2.db	marmbrus
diff --git a/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f
+++ b/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 b/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68
+++ b/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/create_union_table-0-8e765b54f15b948fc88392da69da283 b/sql/hive/src/test/resources/golden/date_2-2-efdf7f5d9f15edcb59a30f8ea166fbf1
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_union_table-0-8e765b54f15b948fc88392da69da283
rename to sql/hive/src/test/resources/golden/date_2-2-efdf7f5d9f15edcb59a30f8ea166fbf1
diff --git a/sql/hive/src/test/resources/golden/create_union_table-1-aee4ce62fc2631423af0f569f4448353 b/sql/hive/src/test/resources/golden/date_3-3-c26f0641e7cec1093273b258e6bf7120
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_union_table-1-aee4ce62fc2631423af0f569f4448353
rename to sql/hive/src/test/resources/golden/date_3-3-c26f0641e7cec1093273b258e6bf7120
diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/date_4-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/date_4-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/create_union_table-2-b1feb4a197caf28d5223e72e10a91e78 b/sql/hive/src/test/resources/golden/date_4-1-b84f7e931d710dcbe3c5126d998285a8
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_union_table-2-b1feb4a197caf28d5223e72e10a91e78
rename to sql/hive/src/test/resources/golden/date_4-1-b84f7e931d710dcbe3c5126d998285a8
diff --git a/sql/hive/src/test/resources/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 b/sql/hive/src/test/resources/golden/date_4-2-6272f5e518f6a20bc96a5870ff315c4f
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91
rename to sql/hive/src/test/resources/golden/date_4-2-6272f5e518f6a20bc96a5870ff315c4f
diff --git a/sql/hive/src/test/resources/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 b/sql/hive/src/test/resources/golden/date_4-3-4a0e7bde447ef616b98e0f55d2886de0
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92
rename to sql/hive/src/test/resources/golden/date_4-3-4a0e7bde447ef616b98e0f55d2886de0
diff --git a/sql/hive/src/test/resources/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd b/sql/hive/src/test/resources/golden/date_4-4-6c4c2941bae77147a4d3d8fcaa1c88c8
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd
rename to sql/hive/src/test/resources/golden/date_4-4-6c4c2941bae77147a4d3d8fcaa1c88c8
diff --git a/sql/hive/src/test/resources/golden/date_4-5-bee09a7384666043621f68297cee2e68 b/sql/hive/src/test/resources/golden/date_4-5-bee09a7384666043621f68297cee2e68
new file mode 100644
index 0000000000000000000000000000000000000000..b61affde4ffced92ae34395dde581139433e82b5
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/date_4-5-bee09a7384666043621f68297cee2e68
@@ -0,0 +1 @@
+2011-01-01	2011-01-01
diff --git a/sql/hive/src/test/resources/golden/create_view-11-2021c047203276dd2db51a56e672fcea b/sql/hive/src/test/resources/golden/date_4-6-b84f7e931d710dcbe3c5126d998285a8
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-11-2021c047203276dd2db51a56e672fcea
rename to sql/hive/src/test/resources/golden/date_4-6-b84f7e931d710dcbe3c5126d998285a8
diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/date_comparison-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/date_comparison-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05 b/sql/hive/src/test/resources/golden/date_comparison-1-69eec445bd045c9dc899fafa348d8495
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05
rename to sql/hive/src/test/resources/golden/date_comparison-1-69eec445bd045c9dc899fafa348d8495
diff --git a/sql/hive/src/test/resources/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 b/sql/hive/src/test/resources/golden/date_comparison-10-bcd987341fc1c38047a27d29dac6ae7c
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575
rename to sql/hive/src/test/resources/golden/date_comparison-10-bcd987341fc1c38047a27d29dac6ae7c
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc b/sql/hive/src/test/resources/golden/date_comparison-11-a9f2560c273163e11306d4f1dd1d9d54
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc
rename to sql/hive/src/test/resources/golden/date_comparison-11-a9f2560c273163e11306d4f1dd1d9d54
diff --git a/sql/hive/src/test/resources/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 b/sql/hive/src/test/resources/golden/date_comparison-12-4a7bac9ddcf40db6329faaec8e426543
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439
rename to sql/hive/src/test/resources/golden/date_comparison-12-4a7bac9ddcf40db6329faaec8e426543
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-3-13e17ed811165196416f777cbc162592 b/sql/hive/src/test/resources/golden/date_comparison-2-fcc400871a502009c8680509e3869ec1
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_comparison-3-13e17ed811165196416f777cbc162592
rename to sql/hive/src/test/resources/golden/date_comparison-2-fcc400871a502009c8680509e3869ec1
diff --git a/sql/hive/src/test/resources/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 b/sql/hive/src/test/resources/golden/date_comparison-3-b8598a4d0c948c2ddcf3eeef0abf2264
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439
rename to sql/hive/src/test/resources/golden/date_comparison-3-b8598a4d0c948c2ddcf3eeef0abf2264
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f b/sql/hive/src/test/resources/golden/date_comparison-4-14d35f266be9cceb11a2ae09ec8b3835
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f
rename to sql/hive/src/test/resources/golden/date_comparison-4-14d35f266be9cceb11a2ae09ec8b3835
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/date_comparison-5-c8865b14d53f2c2496fb69ee8191bf37
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135
rename to sql/hive/src/test/resources/golden/date_comparison-5-c8865b14d53f2c2496fb69ee8191bf37
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/date_comparison-6-f2c907e64da8166a731ddc0ed19bad6c
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135
rename to sql/hive/src/test/resources/golden/date_comparison-6-f2c907e64da8166a731ddc0ed19bad6c
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-10-51822ac740629bebd81d2abda6e1144 b/sql/hive/src/test/resources/golden/date_comparison-7-5606505a92bad10023ad9a3ef77eacc9
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-10-51822ac740629bebd81d2abda6e1144
rename to sql/hive/src/test/resources/golden/date_comparison-7-5606505a92bad10023ad9a3ef77eacc9
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/date_comparison-8-47913d4aaf0d468ab3764cc3bfd68eb
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135
rename to sql/hive/src/test/resources/golden/date_comparison-8-47913d4aaf0d468ab3764cc3bfd68eb
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-11-441306cae24618c49ec63445a31bf16b b/sql/hive/src/test/resources/golden/date_comparison-9-1e5ce4f833b6fba45618437c8fb7643c
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-11-441306cae24618c49ec63445a31bf16b
rename to sql/hive/src/test/resources/golden/date_comparison-9-1e5ce4f833b6fba45618437c8fb7643c
diff --git a/sql/hive/src/test/resources/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc b/sql/hive/src/test/resources/golden/date_join1-2-e967e1ef6b209dfa5bdc60021dcb1964
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc
rename to sql/hive/src/test/resources/golden/date_join1-2-e967e1ef6b209dfa5bdc60021dcb1964
diff --git a/sql/hive/src/test/resources/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a b/sql/hive/src/test/resources/golden/date_serde-7-580096b3b48db26bea91b80e1e1b081a
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a
rename to sql/hive/src/test/resources/golden/date_serde-7-580096b3b48db26bea91b80e1e1b081a
diff --git a/sql/hive/src/test/resources/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 b/sql/hive/src/test/resources/golden/date_udf-7-ef82dff775f4aba5d7a638b4e5fd9c5d
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3
rename to sql/hive/src/test/resources/golden/date_udf-7-ef82dff775f4aba5d7a638b4e5fd9c5d
diff --git a/sql/hive/src/test/resources/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 b/sql/hive/src/test/resources/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1
deleted file mode 100644
index c270c7cbdfa1fb8a8a95d2196e0e7b2bc01bb9a0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1
+++ /dev/null
@@ -1,4 +0,0 @@
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-Detailed Table Information	Table(tableName:t1, dbName:default, owner:marmbrus, createTime:1392063041, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392063041}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 b/sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75
deleted file mode 100644
index 53aca7545dac72a65fa4e5c10be5c2cf85ac34b9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75
+++ /dev/null
@@ -1 +0,0 @@
-17.29
diff --git a/sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 b/sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937
deleted file mode 100644
index 53aca7545dac72a65fa4e5c10be5c2cf85ac34b9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937
+++ /dev/null
@@ -1 +0,0 @@
-17.29
diff --git a/sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 b/sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989
deleted file mode 100644
index c4a17c1b14c889ef9120d4261dff9322662f432b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989
+++ /dev/null
@@ -1 +0,0 @@
-1969-12-31 16:00:17.29
diff --git a/sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 b/sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40
deleted file mode 100644
index 98d9bcb75a685dfbfd60f611c309410152935b3d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40
+++ /dev/null
@@ -1 +0,0 @@
-17
diff --git a/sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b b/sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b
deleted file mode 100644
index 98d9bcb75a685dfbfd60f611c309410152935b3d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b
+++ /dev/null
@@ -1 +0,0 @@
-17
diff --git a/sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a b/sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a
deleted file mode 100644
index 98d9bcb75a685dfbfd60f611c309410152935b3d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a
+++ /dev/null
@@ -1 +0,0 @@
-17
diff --git a/sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d b/sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d
deleted file mode 100644
index 98d9bcb75a685dfbfd60f611c309410152935b3d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d
+++ /dev/null
@@ -1 +0,0 @@
-17
diff --git a/sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c b/sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c
deleted file mode 100644
index 53aca7545dac72a65fa4e5c10be5c2cf85ac34b9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c
+++ /dev/null
@@ -1 +0,0 @@
-17.29
diff --git a/sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 b/sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92
deleted file mode 100644
index 53aca7545dac72a65fa4e5c10be5c2cf85ac34b9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92
+++ /dev/null
@@ -1 +0,0 @@
-17.29
diff --git a/sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 b/sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50
deleted file mode 100644
index 53aca7545dac72a65fa4e5c10be5c2cf85ac34b9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50
+++ /dev/null
@@ -1 +0,0 @@
-17.29
diff --git a/sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 b/sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0
deleted file mode 100644
index b1bd38b62a0800a4f6a80c34e21c5acffae52c7e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0
+++ /dev/null
@@ -1 +0,0 @@
-13
diff --git a/sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 b/sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02
deleted file mode 100644
index de7771ac2357045d628a409fb26962c8f322a32f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02
+++ /dev/null
@@ -1 +0,0 @@
--3827
diff --git a/sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 b/sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580
deleted file mode 100644
index 272791f4022503b499f3eac30bdd9e72937c23bb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580
+++ /dev/null
@@ -1 +0,0 @@
-3404045
diff --git a/sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d b/sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d
deleted file mode 100644
index 272791f4022503b499f3eac30bdd9e72937c23bb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d
+++ /dev/null
@@ -1 +0,0 @@
-3404045
diff --git a/sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e b/sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e
deleted file mode 100644
index deb8427800ee49e849b778dedc25bf52a6bc022f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e
+++ /dev/null
@@ -1 +0,0 @@
-3404045.5
diff --git a/sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 b/sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92
deleted file mode 100644
index 6f31e8fe550343d55be1df62a2f3aa52b94de442..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92
+++ /dev/null
@@ -1 +0,0 @@
-3404045.5044003
diff --git a/sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 b/sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50
deleted file mode 100644
index 6f31e8fe550343d55be1df62a2f3aa52b94de442..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50
+++ /dev/null
@@ -1 +0,0 @@
-3404045.5044003
diff --git a/sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 b/sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821
deleted file mode 100644
index 6324d401a069f4020efcf0ff07442724b52f47c2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821
+++ /dev/null
@@ -1 +0,0 @@
-3.14
diff --git a/sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 b/sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8
deleted file mode 100644
index 6324d401a069f4020efcf0ff07442724b52f47c2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8
+++ /dev/null
@@ -1 +0,0 @@
-3.14
diff --git a/sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff b/sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff
deleted file mode 100644
index 603f18cc37bc446251da5101fd133ba926e208c4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff
+++ /dev/null
@@ -1 +0,0 @@
-1355944339.1234567
diff --git a/sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 b/sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6
deleted file mode 100644
index 00750edc07d6415dcc07ae0351e9397b0222b7ba..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6
+++ /dev/null
@@ -1 +0,0 @@
-3
diff --git a/sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 b/sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1
deleted file mode 100644
index 00750edc07d6415dcc07ae0351e9397b0222b7ba..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1
+++ /dev/null
@@ -1 +0,0 @@
-3
diff --git a/sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a b/sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a
deleted file mode 100644
index 474c8b180aea9c1402d7d5ff6800c49e9c7a8cf7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a
+++ /dev/null
@@ -1 +0,0 @@
-0.99999999999999999999
diff --git a/sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 b/sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0
deleted file mode 100644
index 98d9bcb75a685dfbfd60f611c309410152935b3d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0
+++ /dev/null
@@ -1 +0,0 @@
-17
diff --git a/sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 b/sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02
deleted file mode 100644
index 98d9bcb75a685dfbfd60f611c309410152935b3d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02
+++ /dev/null
@@ -1 +0,0 @@
-17
diff --git a/sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 b/sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580
deleted file mode 100644
index 98d9bcb75a685dfbfd60f611c309410152935b3d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580
+++ /dev/null
@@ -1 +0,0 @@
-17
diff --git a/sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d b/sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d
deleted file mode 100644
index 98d9bcb75a685dfbfd60f611c309410152935b3d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d
+++ /dev/null
@@ -1 +0,0 @@
-17
diff --git a/sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e b/sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e
deleted file mode 100644
index 53aca7545dac72a65fa4e5c10be5c2cf85ac34b9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e
+++ /dev/null
@@ -1 +0,0 @@
-17.29
diff --git a/sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 b/sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624
deleted file mode 100644
index 8d8753f153d7c9a836f576cf189659082e1b52ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624
+++ /dev/null
@@ -1,4 +0,0 @@
-3.14	3
-3.14	3
-3.14	3
-3.14	4
diff --git a/sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 b/sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15
deleted file mode 100644
index 8d8753f153d7c9a836f576cf189659082e1b52ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15
+++ /dev/null
@@ -1,4 +0,0 @@
-3.14	3
-3.14	3
-3.14	3
-3.14	4
diff --git a/sql/hive/src/test/resources/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b b/sql/hive/src/test/resources/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b
deleted file mode 100644
index 3e290231c27e2bece9e08112cb7fcb94bddcf9f6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b
+++ /dev/null
@@ -1,38 +0,0 @@
-NULL	0
-NULL	0
--1234567890.123456789	-1234567890
--4400	4400
--1255.49	-1255
--1.122	-11
--1.12	-1
--1.12	-1
--0.333	0
--0.33	0
--0.3	0
-0	0
-0	0
-0.01	0
-0.02	0
-0.1	0
-0.2	0
-0.3	0
-0.33	0
-0.333	0
-0.9999999999999999999999999	1
-1	1
-1	1
-1.12	1
-1.122	1
-2	2
-2	2
-3.14	3
-3.14	3
-3.14	3
-3.14	4
-10	10
-20	20
-100	100
-124	124
-125.2	125
-200	200
-1234567890.12345678	1234567890
diff --git a/sql/hive/src/test/resources/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 b/sql/hive/src/test/resources/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263
deleted file mode 100644
index 64fa7bca9a81b8d4a88dc32d7bdc9c026803d1b4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263
+++ /dev/null
@@ -1,38 +0,0 @@
-1234567890.12345678	1234567890
-200	200
-125.2	125
-124	124
-100	100
-20	20
-10	10
-3.14	4
-3.14	3
-3.14	3
-3.14	3
-2	2
-2	2
-1.122	1
-1.12	1
-1	1
-1	1
-0.9999999999999999999999999	1
-0.333	0
-0.33	0
-0.3	0
-0.2	0
-0.1	0
-0.02	0
-0.01	0
-0	0
-0	0
--0.3	0
--0.33	0
--0.333	0
--1.12	-1
--1.12	-1
--1.122	-11
--1255.49	-1255
--4400	4400
--1234567890.123456789	-1234567890
-NULL	0
-NULL	0
diff --git a/sql/hive/src/test/resources/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b b/sql/hive/src/test/resources/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b
deleted file mode 100644
index 3e290231c27e2bece9e08112cb7fcb94bddcf9f6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b
+++ /dev/null
@@ -1,38 +0,0 @@
-NULL	0
-NULL	0
--1234567890.123456789	-1234567890
--4400	4400
--1255.49	-1255
--1.122	-11
--1.12	-1
--1.12	-1
--0.333	0
--0.33	0
--0.3	0
-0	0
-0	0
-0.01	0
-0.02	0
-0.1	0
-0.2	0
-0.3	0
-0.33	0
-0.333	0
-0.9999999999999999999999999	1
-1	1
-1	1
-1.12	1
-1.122	1
-2	2
-2	2
-3.14	3
-3.14	3
-3.14	3
-3.14	4
-10	10
-20	20
-100	100
-124	124
-125.2	125
-200	200
-1234567890.12345678	1234567890
diff --git a/sql/hive/src/test/resources/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 b/sql/hive/src/test/resources/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55
deleted file mode 100644
index 24d34ee5d8c1b596725c0a5dae9d95a0c5feca74..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55
+++ /dev/null
@@ -1,30 +0,0 @@
-NULL
--1234567890.123456789
--4400
--1255.49
--1.122
--1.12
--0.333
--0.33
--0.3
-0
-0.01
-0.02
-0.1
-0.2
-0.3
-0.33
-0.333
-0.9999999999999999999999999
-1
-1.12
-1.122
-2
-3.14
-10
-20
-100
-124
-125.2
-200
-1234567890.12345678
diff --git a/sql/hive/src/test/resources/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 b/sql/hive/src/test/resources/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052
deleted file mode 100644
index e08f588c894617651a1e777d0bd5a32fb2d9196c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052
+++ /dev/null
@@ -1,30 +0,0 @@
-NULL	0
--1234567890.123456789	-1234567890
--4400	4400
--1255.49	-1255
--1.122	-11
--1.12	-2
--0.333	0
--0.33	0
--0.3	0
-0	0
-0.01	0
-0.02	0
-0.1	0
-0.2	0
-0.3	0
-0.33	0
-0.333	0
-0.9999999999999999999999999	1
-1	2
-1.12	1
-1.122	1
-2	4
-3.14	13
-10	10
-20	20
-100	100
-124	124
-125.2	125
-200	200
-1234567890.12345678	1234567890
diff --git a/sql/hive/src/test/resources/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 b/sql/hive/src/test/resources/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800
deleted file mode 100644
index 796707d06b0ddab16f1808cf20bb2752c332f79c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800
+++ /dev/null
@@ -1,17 +0,0 @@
--1234567890	-1234567890.123456789
--1255	-1255.49
--11	-1.122
--1	-2.24
-0	0.33
-1	5.2419999999999999999999999
-2	4
-3	9.42
-4	3.14
-10	10
-20	20
-100	100
-124	124
-125	125.2
-200	200
-4400	-4400
-1234567890	1234567890.12345678
diff --git a/sql/hive/src/test/resources/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 b/sql/hive/src/test/resources/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4
deleted file mode 100644
index 4217ad848170e6e83e4b92e01becb212fe0904a3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4
+++ /dev/null
@@ -1,56 +0,0 @@
--1234567890.123456789	-1234567890	-1234567890.123456789	-1234567890
--4400	4400	-4400	4400
--1255.49	-1255	-1255.49	-1255
--1.122	-11	-1.122	-11
--1.12	-1	-1.12	-1
--1.12	-1	-1.12	-1
--1.12	-1	-1.12	-1
--1.12	-1	-1.12	-1
--0.333	0	-0.333	0
--0.33	0	-0.33	0
--0.3	0	-0.3	0
-0	0	0	0
-0	0	0	0
-0	0	0	0
-0	0	0	0
-0.01	0	0.01	0
-0.02	0	0.02	0
-0.1	0	0.1	0
-0.2	0	0.2	0
-0.3	0	0.3	0
-0.33	0	0.33	0
-0.333	0	0.333	0
-0.9999999999999999999999999	1	0.9999999999999999999999999	1
-1	1	1	1
-1	1	1	1
-1	1	1	1
-1	1	1	1
-1.12	1	1.12	1
-1.122	1	1.122	1
-2	2	2	2
-2	2	2	2
-2	2	2	2
-2	2	2	2
-3.14	3	3.14	3
-3.14	3	3.14	3
-3.14	3	3.14	3
-3.14	3	3.14	3
-3.14	3	3.14	3
-3.14	3	3.14	3
-3.14	3	3.14	3
-3.14	3	3.14	3
-3.14	3	3.14	3
-3.14	3	3.14	4
-3.14	3	3.14	4
-3.14	3	3.14	4
-3.14	4	3.14	3
-3.14	4	3.14	3
-3.14	4	3.14	3
-3.14	4	3.14	4
-10	10	10	10
-20	20	20	20
-100	100	100	100
-124	124	124	124
-125.2	125	125.2	125
-200	200	200	200
-1234567890.12345678	1234567890	1234567890.12345678	1234567890
diff --git a/sql/hive/src/test/resources/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 b/sql/hive/src/test/resources/golden/decimal_4-2-85c3185beb011f5c1e6856fc773a7484
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447
rename to sql/hive/src/test/resources/golden/decimal_4-2-85c3185beb011f5c1e6856fc773a7484
diff --git a/sql/hive/src/test/resources/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 b/sql/hive/src/test/resources/golden/decimal_4-3-1451d7491441c1632fd5f751876cce6e
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4
rename to sql/hive/src/test/resources/golden/decimal_4-3-1451d7491441c1632fd5f751876cce6e
diff --git a/sql/hive/src/test/resources/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a b/sql/hive/src/test/resources/golden/decimal_4-4-1bf9ff1d72a06c33885ba695adf2511d
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a
rename to sql/hive/src/test/resources/golden/decimal_4-4-1bf9ff1d72a06c33885ba695adf2511d
diff --git a/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e b/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e
index 3e290231c27e2bece9e08112cb7fcb94bddcf9f6..f59549a6e4a46db028051007f015457d8ba62c48 100644
--- a/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e
+++ b/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e
@@ -1,5 +1,4 @@
 NULL	0
-NULL	0
 -1234567890.123456789	-1234567890
 -4400	4400
 -1255.49	-1255
@@ -11,6 +10,7 @@ NULL	0
 -0.3	0
 0	0
 0	0
+0	0
 0.01	0
 0.02	0
 0.1	0
diff --git a/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 b/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531
index 795a4b567ab7f3d4c0494958ce90c07b2efea533..6bada475c6d3d55585599ef346ca2f57b1f1f482 100644
--- a/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531
+++ b/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531
@@ -1,5 +1,4 @@
 NULL	NULL
-NULL	NULL
 -1234567890.123456789	-3703703670.370370367
 -4400	-13200
 -1255.49	-3766.47
@@ -11,6 +10,7 @@ NULL	NULL
 -0.3	-0.9
 0	0
 0	0
+0	0
 0.01	0.03
 0.02	0.06
 0.1	0.3
diff --git a/sql/hive/src/test/resources/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 b/sql/hive/src/test/resources/golden/decimal_join-0-4668e9dee2cd7a32f2b7311d7cd35508
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6
rename to sql/hive/src/test/resources/golden/decimal_join-0-4668e9dee2cd7a32f2b7311d7cd35508
diff --git a/sql/hive/src/test/resources/golden/create_view-2-d80dcd1271ab264292e9938f3162427c b/sql/hive/src/test/resources/golden/decimal_join-1-5098974222b22a21ed847c7906df9313
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-2-d80dcd1271ab264292e9938f3162427c
rename to sql/hive/src/test/resources/golden/decimal_join-1-5098974222b22a21ed847c7906df9313
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b b/sql/hive/src/test/resources/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b
deleted file mode 100644
index 3d9e792183f3cb86c69b6053d035d8d0fb250592..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b
+++ /dev/null
@@ -1,75 +0,0 @@
-NULL
-NULL
-NULL
-NULL
-NULL
-NULL
-NULL
-NULL
--99999999999999999999999999999999999999
--999999999999999999999999999999999999
--99999999999999999999999999999999999
--0.0000000000000000000000000000000000001
-0
-0.0000000000000000000000000000000000001
-0.123456789012345
-0.1234567890123456789012345678901234578
-1.234567890123456
-1.2345678901234567890123456789012345678
-12.34567890123456
-12.345678901234567890123456789012345678
-123.4567890123456
-123.45678901234567890123456789012345678
-1234.567890123456
-1234.5678901234567890123456789012345678
-12345.67890123456
-12345.678901234567890123456789012345678
-123456.7890123456
-123456.78901234567890123456789012345678
-1234567.890123456
-1234567.8901234567890123456789012345678
-12345678.90123456
-12345678.901234567890123456789012345678
-123456789.0123456
-123456789.01234567890123456789012345678
-1234567890.123456
-1234567890.1234567890123456789012345678
-12345678901.23456
-12345678901.234567890123456789012345678
-123456789012.3456
-123456789012.34567890123456789012345678
-1234567890123.456
-1234567890123.4567890123456789012345678
-12345678901234.56
-12345678901234.567890123456789012345678
-123456789012345.6
-123456789012345.67890123456789012345678
-1234567890123456.7890123456789012345678
-12345678901234567.890123456789012345678
-123456789012345678.90123456789012345678
-1234567890123456789.0123456789012345678
-12345678901234567890.123456789012345678
-123456789012345678901.23456789012345678
-1234567890123456789012.3456789012345678
-12345678901234567890123.456789012345678
-123456789012345678901234.56789012345678
-1234567890123456789012345.6789012345678
-12345678901234567890123456.789012345678
-123456789012345678901234567.89012345678
-1234567890123456789012345678.9012345678
-12345678901234567890123456789.012345678
-123456789012345678901234567890.12345678
-1234567890123456789012345678901.2345678
-12345678901234567890123456789012.345678
-123456789012345678901234567890123.45678
-1234567890123456789012345678901234.5678
-12345678901234567890123456789012345.678
-99999999999999999999999999999999999
-123456789012345678901234567890123456.78
-999999999999999999999999999999999999
-12345678901234567890123456789012345678
-12345678901234567890123456789012345678
-12345678901234567890123456789012345678
-12345678901234567890123456789012345678
-12345678901234567890123456789012345678
-99999999999999999999999999999999999999
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc b/sql/hive/src/test/resources/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc
deleted file mode 100644
index 9853ce72ed8c34e2b25e8fb690ba187a157efaec..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc
+++ /dev/null
@@ -1,75 +0,0 @@
-NULL	NULL	NULL
-NULL	NULL	NULL
-NULL	NULL	NULL
-NULL	NULL	NULL
-NULL	NULL	NULL
-NULL	NULL	NULL
-NULL	NULL	NULL
-NULL	NULL	NULL
--99999999999999999999999999999999999999	-99999999999999999999999999999999999998	NULL
--999999999999999999999999999999999999	-999999999999999999999999999999999998	-1000000000000000000000000000000000000
--99999999999999999999999999999999999	-99999999999999999999999999999999998	-100000000000000000000000000000000000
--0.0000000000000000000000000000000000001	0.9999999999999999999999999999999999999	-1.0000000000000000000000000000000000001
-0	1	-1
-0.0000000000000000000000000000000000001	1.0000000000000000000000000000000000001	-0.9999999999999999999999999999999999999
-0.123456789012345	1.123456789012345	-0.876543210987655
-0.1234567890123456789012345678901234578	1.1234567890123456789012345678901234578	-0.8765432109876543210987654321098765422
-1.234567890123456	2.234567890123456	0.234567890123456
-1.2345678901234567890123456789012345678	2.2345678901234567890123456789012345678	0.2345678901234567890123456789012345678
-12.34567890123456	13.34567890123456	11.34567890123456
-12.345678901234567890123456789012345678	13.345678901234567890123456789012345678	11.345678901234567890123456789012345678
-123.4567890123456	124.4567890123456	122.4567890123456
-123.45678901234567890123456789012345678	124.45678901234567890123456789012345678	122.45678901234567890123456789012345678
-1234.567890123456	1235.567890123456	1233.567890123456
-1234.5678901234567890123456789012345678	1235.5678901234567890123456789012345678	1233.5678901234567890123456789012345678
-12345.67890123456	12346.67890123456	12344.67890123456
-12345.678901234567890123456789012345678	12346.678901234567890123456789012345678	12344.678901234567890123456789012345678
-123456.7890123456	123457.7890123456	123455.7890123456
-123456.78901234567890123456789012345678	123457.78901234567890123456789012345678	123455.78901234567890123456789012345678
-1234567.890123456	1234568.890123456	1234566.890123456
-1234567.8901234567890123456789012345678	1234568.8901234567890123456789012345678	1234566.8901234567890123456789012345678
-12345678.90123456	12345679.90123456	12345677.90123456
-12345678.901234567890123456789012345678	12345679.901234567890123456789012345678	12345677.901234567890123456789012345678
-123456789.0123456	123456790.0123456	123456788.0123456
-123456789.01234567890123456789012345678	123456790.01234567890123456789012345678	123456788.01234567890123456789012345678
-1234567890.123456	1234567891.123456	1234567889.123456
-1234567890.1234567890123456789012345678	1234567891.1234567890123456789012345678	1234567889.1234567890123456789012345678
-12345678901.23456	12345678902.23456	12345678900.23456
-12345678901.234567890123456789012345678	12345678902.234567890123456789012345678	12345678900.234567890123456789012345678
-123456789012.3456	123456789013.3456	123456789011.3456
-123456789012.34567890123456789012345678	123456789013.34567890123456789012345678	123456789011.34567890123456789012345678
-1234567890123.456	1234567890124.456	1234567890122.456
-1234567890123.4567890123456789012345678	1234567890124.4567890123456789012345678	1234567890122.4567890123456789012345678
-12345678901234.56	12345678901235.56	12345678901233.56
-12345678901234.567890123456789012345678	12345678901235.567890123456789012345678	12345678901233.567890123456789012345678
-123456789012345.6	123456789012346.6	123456789012344.6
-123456789012345.67890123456789012345678	123456789012346.67890123456789012345678	123456789012344.67890123456789012345678
-1234567890123456.7890123456789012345678	1234567890123457.7890123456789012345678	1234567890123455.7890123456789012345678
-12345678901234567.890123456789012345678	12345678901234568.890123456789012345678	12345678901234566.890123456789012345678
-123456789012345678.90123456789012345678	123456789012345679.90123456789012345678	123456789012345677.90123456789012345678
-1234567890123456789.0123456789012345678	1234567890123456790.0123456789012345678	1234567890123456788.0123456789012345678
-12345678901234567890.123456789012345678	12345678901234567891.123456789012345678	12345678901234567889.123456789012345678
-123456789012345678901.23456789012345678	123456789012345678902.23456789012345678	123456789012345678900.23456789012345678
-1234567890123456789012.3456789012345678	1234567890123456789013.3456789012345678	1234567890123456789011.3456789012345678
-12345678901234567890123.456789012345678	12345678901234567890124.456789012345678	12345678901234567890122.456789012345678
-123456789012345678901234.56789012345678	123456789012345678901235.56789012345678	123456789012345678901233.56789012345678
-1234567890123456789012345.6789012345678	1234567890123456789012346.6789012345678	1234567890123456789012344.6789012345678
-12345678901234567890123456.789012345678	12345678901234567890123457.789012345678	12345678901234567890123455.789012345678
-123456789012345678901234567.89012345678	123456789012345678901234568.89012345678	123456789012345678901234566.89012345678
-1234567890123456789012345678.9012345678	1234567890123456789012345679.9012345678	1234567890123456789012345677.9012345678
-12345678901234567890123456789.012345678	12345678901234567890123456790.012345678	12345678901234567890123456788.012345678
-123456789012345678901234567890.12345678	123456789012345678901234567891.12345678	123456789012345678901234567889.12345678
-1234567890123456789012345678901.2345678	1234567890123456789012345678902.2345678	1234567890123456789012345678900.2345678
-12345678901234567890123456789012.345678	12345678901234567890123456789013.345678	12345678901234567890123456789011.345678
-123456789012345678901234567890123.45678	123456789012345678901234567890124.45678	123456789012345678901234567890122.45678
-1234567890123456789012345678901234.5678	1234567890123456789012345678901235.5678	1234567890123456789012345678901233.5678
-12345678901234567890123456789012345.678	12345678901234567890123456789012346.678	12345678901234567890123456789012344.678
-99999999999999999999999999999999999	100000000000000000000000000000000000	99999999999999999999999999999999998
-123456789012345678901234567890123456.78	123456789012345678901234567890123457.78	123456789012345678901234567890123455.78
-999999999999999999999999999999999999	1000000000000000000000000000000000000	999999999999999999999999999999999998
-12345678901234567890123456789012345678	12345678901234567890123456789012345679	12345678901234567890123456789012345677
-12345678901234567890123456789012345678	12345678901234567890123456789012345679	12345678901234567890123456789012345677
-12345678901234567890123456789012345678	12345678901234567890123456789012345679	12345678901234567890123456789012345677
-12345678901234567890123456789012345678	12345678901234567890123456789012345679	12345678901234567890123456789012345677
-12345678901234567890123456789012345678	12345678901234567890123456789012345679	12345678901234567890123456789012345677
-99999999999999999999999999999999999999	NULL	99999999999999999999999999999999999998
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef b/sql/hive/src/test/resources/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef
deleted file mode 100644
index 7cc75c789dee1d1bb6a5e4b1c660d14dc97fd1bd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef
+++ /dev/null
@@ -1,75 +0,0 @@
-NULL	NULL	NULL
-NULL	NULL	NULL
-NULL	NULL	NULL
-NULL	NULL	NULL
-NULL	NULL	NULL
-NULL	NULL	NULL
-NULL	NULL	NULL
-NULL	NULL	NULL
--99999999999999999999999999999999999999	NULL	-33333333333333333333333333333333333333
--999999999999999999999999999999999999	-1999999999999999999999999999999999998	-333333333333333333333333333333333333
--99999999999999999999999999999999999	-199999999999999999999999999999999998	-33333333333333333333333333333333333
--0.0000000000000000000000000000000000001	-0.0000000000000000000000000000000000002	0
-0	0	0
-0.0000000000000000000000000000000000001	0.0000000000000000000000000000000000002	0
-0.123456789012345	0.24691357802469	0.041152263004115
-0.1234567890123456789012345678901234578	0.2469135780246913578024691357802469156	0.0411522630041152263004115226300411526
-1.234567890123456	2.469135780246912	0.411522630041152
-1.2345678901234567890123456789012345678	2.4691357802469135780246913578024691356	0.4115226300411522630041152263004115226
-12.34567890123456	24.69135780246912	4.11522630041152
-12.345678901234567890123456789012345678	24.691357802469135780246913578024691356	4.115226300411522630041152263004115226
-123.4567890123456	246.9135780246912	41.1522630041152
-123.45678901234567890123456789012345678	246.91357802469135780246913578024691356	41.15226300411522630041152263004115226
-1234.567890123456	2469.135780246912	411.522630041152
-1234.5678901234567890123456789012345678	2469.1357802469135780246913578024691356	411.5226300411522630041152263004115226
-12345.67890123456	24691.35780246912	4115.22630041152
-12345.678901234567890123456789012345678	24691.357802469135780246913578024691356	4115.226300411522630041152263004115226
-123456.7890123456	246913.5780246912	41152.2630041152
-123456.78901234567890123456789012345678	246913.57802469135780246913578024691356	41152.26300411522630041152263004115226
-1234567.890123456	2469135.780246912	411522.630041152
-1234567.8901234567890123456789012345678	2469135.7802469135780246913578024691356	411522.6300411522630041152263004115226
-12345678.90123456	24691357.80246912	4115226.30041152
-12345678.901234567890123456789012345678	24691357.802469135780246913578024691356	4115226.300411522630041152263004115226
-123456789.0123456	246913578.0246912	41152263.0041152
-123456789.01234567890123456789012345678	246913578.02469135780246913578024691356	41152263.00411522630041152263004115226
-1234567890.123456	2469135780.246912	411522630.041152
-1234567890.1234567890123456789012345678	2469135780.2469135780246913578024691356	411522630.0411522630041152263004115226
-12345678901.23456	24691357802.46912	4115226300.41152
-12345678901.234567890123456789012345678	24691357802.469135780246913578024691356	4115226300.411522630041152263004115226
-123456789012.3456	246913578024.6912	41152263004.1152
-123456789012.34567890123456789012345678	246913578024.69135780246913578024691356	41152263004.11522630041152263004115226
-1234567890123.456	2469135780246.912	411522630041.152
-1234567890123.4567890123456789012345678	2469135780246.9135780246913578024691356	411522630041.1522630041152263004115226
-12345678901234.56	24691357802469.12	4115226300411.52
-12345678901234.567890123456789012345678	24691357802469.135780246913578024691356	4115226300411.522630041152263004115226
-123456789012345.6	246913578024691.2	41152263004115.2
-123456789012345.67890123456789012345678	246913578024691.35780246913578024691356	41152263004115.22630041152263004115226
-1234567890123456.7890123456789012345678	2469135780246913.5780246913578024691356	411522630041152.2630041152263004115226
-12345678901234567.890123456789012345678	24691357802469135.780246913578024691356	4115226300411522.630041152263004115226
-123456789012345678.90123456789012345678	246913578024691357.80246913578024691356	41152263004115226.30041152263004115226
-1234567890123456789.0123456789012345678	2469135780246913578.0246913578024691356	411522630041152263.0041152263004115226
-12345678901234567890.123456789012345678	24691357802469135780.246913578024691356	4115226300411522630.041152263004115226
-123456789012345678901.23456789012345678	246913578024691357802.46913578024691356	41152263004115226300.41152263004115226
-1234567890123456789012.3456789012345678	2469135780246913578024.6913578024691356	411522630041152263004.1152263004115226
-12345678901234567890123.456789012345678	24691357802469135780246.913578024691356	4115226300411522630041.152263004115226
-123456789012345678901234.56789012345678	246913578024691357802469.13578024691356	41152263004115226300411.52263004115226
-1234567890123456789012345.6789012345678	2469135780246913578024691.3578024691356	411522630041152263004115.2263004115226
-12345678901234567890123456.789012345678	24691357802469135780246913.578024691356	4115226300411522630041152.263004115226
-123456789012345678901234567.89012345678	246913578024691357802469135.78024691356	41152263004115226300411522.63004115226
-1234567890123456789012345678.9012345678	2469135780246913578024691357.8024691356	411522630041152263004115226.3004115226
-12345678901234567890123456789.012345678	24691357802469135780246913578.024691356	4115226300411522630041152263.004115226
-123456789012345678901234567890.12345678	246913578024691357802469135780.24691356	41152263004115226300411522630.04115226
-1234567890123456789012345678901.2345678	2469135780246913578024691357802.4691356	411522630041152263004115226300.4115226
-12345678901234567890123456789012.345678	24691357802469135780246913578024.691356	4115226300411522630041152263004.115226
-123456789012345678901234567890123.45678	246913578024691357802469135780246.91356	41152263004115226300411522630041.15226
-1234567890123456789012345678901234.5678	2469135780246913578024691357802469.1356	411522630041152263004115226300411.5226
-12345678901234567890123456789012345.678	24691357802469135780246913578024691.356	4115226300411522630041152263004115.226
-99999999999999999999999999999999999	199999999999999999999999999999999998	33333333333333333333333333333333333
-123456789012345678901234567890123456.78	246913578024691357802469135780246913.56	41152263004115226300411522630041152.26
-999999999999999999999999999999999999	1999999999999999999999999999999999998	333333333333333333333333333333333333
-12345678901234567890123456789012345678	24691357802469135780246913578024691356	4115226300411522630041152263004115226
-12345678901234567890123456789012345678	24691357802469135780246913578024691356	4115226300411522630041152263004115226
-12345678901234567890123456789012345678	24691357802469135780246913578024691356	4115226300411522630041152263004115226
-12345678901234567890123456789012345678	24691357802469135780246913578024691356	4115226300411522630041152263004115226
-12345678901234567890123456789012345678	24691357802469135780246913578024691356	4115226300411522630041152263004115226
-99999999999999999999999999999999999999	NULL	33333333333333333333333333333333333333
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 b/sql/hive/src/test/resources/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21
deleted file mode 100644
index c40875630d1b26b8bef74b19138aaced7a5d192e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21
+++ /dev/null
@@ -1,75 +0,0 @@
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
--99999999999999999999999999999999999999	-11111111111111111111111111111111111111
--999999999999999999999999999999999999	-111111111111111111111111111111111111
--99999999999999999999999999999999999	-11111111111111111111111111111111111
--0.0000000000000000000000000000000000001	0
-0	0
-0.0000000000000000000000000000000000001	0
-0.123456789012345	0.0137174210013716666666666666666666667
-0.1234567890123456789012345678901234578	0.0137174210013717421001371742100137175
-1.234567890123456	0.1371742100137173333333333333333333333
-1.2345678901234567890123456789012345678	0.1371742100137174210013717421001371742
-12.34567890123456	1.3717421001371733333333333333333333333
-12.345678901234567890123456789012345678	1.371742100137174210013717421001371742
-123.4567890123456	13.717421001371733333333333333333333333
-123.45678901234567890123456789012345678	13.71742100137174210013717421001371742
-1234.567890123456	137.17421001371733333333333333333333333
-1234.5678901234567890123456789012345678	137.1742100137174210013717421001371742
-12345.67890123456	1371.7421001371733333333333333333333333
-12345.678901234567890123456789012345678	1371.742100137174210013717421001371742
-123456.7890123456	13717.421001371733333333333333333333333
-123456.78901234567890123456789012345678	13717.42100137174210013717421001371742
-1234567.890123456	137174.21001371733333333333333333333333
-1234567.8901234567890123456789012345678	137174.2100137174210013717421001371742
-12345678.90123456	1371742.1001371733333333333333333333333
-12345678.901234567890123456789012345678	1371742.100137174210013717421001371742
-123456789.0123456	13717421.001371733333333333333333333333
-123456789.01234567890123456789012345678	13717421.00137174210013717421001371742
-1234567890.123456	137174210.01371733333333333333333333333
-1234567890.1234567890123456789012345678	137174210.0137174210013717421001371742
-12345678901.23456	1371742100.1371733333333333333333333333
-12345678901.234567890123456789012345678	1371742100.137174210013717421001371742
-123456789012.3456	13717421001.371733333333333333333333333
-123456789012.34567890123456789012345678	13717421001.37174210013717421001371742
-1234567890123.456	137174210013.71733333333333333333333333
-1234567890123.4567890123456789012345678	137174210013.7174210013717421001371742
-12345678901234.56	1371742100137.1733333333333333333333333
-12345678901234.567890123456789012345678	1371742100137.174210013717421001371742
-123456789012345.6	13717421001371.733333333333333333333333
-123456789012345.67890123456789012345678	13717421001371.74210013717421001371742
-1234567890123456.7890123456789012345678	137174210013717.4210013717421001371742
-12345678901234567.890123456789012345678	1371742100137174.210013717421001371742
-123456789012345678.90123456789012345678	13717421001371742.10013717421001371742
-1234567890123456789.0123456789012345678	137174210013717421.0013717421001371742
-12345678901234567890.123456789012345678	1371742100137174210.013717421001371742
-123456789012345678901.23456789012345678	13717421001371742100.13717421001371742
-1234567890123456789012.3456789012345678	137174210013717421001.3717421001371742
-12345678901234567890123.456789012345678	1371742100137174210013.717421001371742
-123456789012345678901234.56789012345678	13717421001371742100137.17421001371742
-1234567890123456789012345.6789012345678	137174210013717421001371.7421001371742
-12345678901234567890123456.789012345678	1371742100137174210013717.421001371742
-123456789012345678901234567.89012345678	13717421001371742100137174.21001371742
-1234567890123456789012345678.9012345678	137174210013717421001371742.1001371742
-12345678901234567890123456789.012345678	1371742100137174210013717421.001371742
-123456789012345678901234567890.12345678	13717421001371742100137174210.01371742
-1234567890123456789012345678901.2345678	137174210013717421001371742100.1371742
-12345678901234567890123456789012.345678	1371742100137174210013717421001.371742
-123456789012345678901234567890123.45678	13717421001371742100137174210013.71742
-1234567890123456789012345678901234.5678	137174210013717421001371742100137.1742
-12345678901234567890123456789012345.678	1371742100137174210013717421001371.742
-99999999999999999999999999999999999	11111111111111111111111111111111111
-123456789012345678901234567890123456.78	13717421001371742100137174210013717.42
-999999999999999999999999999999999999	111111111111111111111111111111111111
-12345678901234567890123456789012345678	1371742100137174210013717421001371742
-12345678901234567890123456789012345678	1371742100137174210013717421001371742
-12345678901234567890123456789012345678	1371742100137174210013717421001371742
-12345678901234567890123456789012345678	1371742100137174210013717421001371742
-12345678901234567890123456789012345678	1371742100137174210013717421001371742
-99999999999999999999999999999999999999	11111111111111111111111111111111111111
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 b/sql/hive/src/test/resources/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7
deleted file mode 100644
index bd23d17293f797ee4993f438ee373a92cbc6d41a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7
+++ /dev/null
@@ -1,75 +0,0 @@
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
--99999999999999999999999999999999999999	-3703703703703703703703703703703703703.7
--999999999999999999999999999999999999	-37037037037037037037037037037037037
--99999999999999999999999999999999999	-3703703703703703703703703703703703.6667
--0.0000000000000000000000000000000000001	0
-0	0
-0.0000000000000000000000000000000000001	0
-0.123456789012345	0.0045724736671238888888888888888888889
-0.1234567890123456789012345678901234578	0.0045724736671239140333790580700045725
-1.234567890123456	0.0457247366712391111111111111111111111
-1.2345678901234567890123456789012345678	0.0457247366712391403337905807000457247
-12.34567890123456	0.4572473667123911111111111111111111111
-12.345678901234567890123456789012345678	0.4572473667123914033379058070004572473
-123.4567890123456	4.5724736671239111111111111111111111111
-123.45678901234567890123456789012345678	4.5724736671239140333790580700045724733
-1234.567890123456	45.724736671239111111111111111111111111
-1234.5678901234567890123456789012345678	45.724736671239140333790580700045724733
-12345.67890123456	457.24736671239111111111111111111111111
-12345.678901234567890123456789012345678	457.24736671239140333790580700045724733
-123456.7890123456	4572.4736671239111111111111111111111111
-123456.78901234567890123456789012345678	4572.4736671239140333790580700045724733
-1234567.890123456	45724.736671239111111111111111111111111
-1234567.8901234567890123456789012345678	45724.736671239140333790580700045724733
-12345678.90123456	457247.36671239111111111111111111111111
-12345678.901234567890123456789012345678	457247.36671239140333790580700045724733
-123456789.0123456	4572473.6671239111111111111111111111111
-123456789.01234567890123456789012345678	4572473.6671239140333790580700045724733
-1234567890.123456	45724736.671239111111111111111111111111
-1234567890.1234567890123456789012345678	45724736.671239140333790580700045724733
-12345678901.23456	457247366.71239111111111111111111111111
-12345678901.234567890123456789012345678	457247366.71239140333790580700045724733
-123456789012.3456	4572473667.1239111111111111111111111111
-123456789012.34567890123456789012345678	4572473667.1239140333790580700045724733
-1234567890123.456	45724736671.239111111111111111111111111
-1234567890123.4567890123456789012345678	45724736671.239140333790580700045724733
-12345678901234.56	457247366712.39111111111111111111111111
-12345678901234.567890123456789012345678	457247366712.39140333790580700045724733
-123456789012345.6	4572473667123.9111111111111111111111111
-123456789012345.67890123456789012345678	4572473667123.9140333790580700045724733
-1234567890123456.7890123456789012345678	45724736671239.140333790580700045724733
-12345678901234567.890123456789012345678	457247366712391.40333790580700045724733
-123456789012345678.90123456789012345678	4572473667123914.0333790580700045724733
-1234567890123456789.0123456789012345678	45724736671239140.333790580700045724733
-12345678901234567890.123456789012345678	457247366712391403.33790580700045724733
-123456789012345678901.23456789012345678	4572473667123914033.3790580700045724733
-1234567890123456789012.3456789012345678	45724736671239140333.790580700045724733
-12345678901234567890123.456789012345678	457247366712391403337.90580700045724733
-123456789012345678901234.56789012345678	4572473667123914033379.0580700045724733
-1234567890123456789012345.6789012345678	45724736671239140333790.580700045724733
-12345678901234567890123456.789012345678	457247366712391403337905.80700045724733
-123456789012345678901234567.89012345678	4572473667123914033379058.0700045724733
-1234567890123456789012345678.9012345678	45724736671239140333790580.700045724733
-12345678901234567890123456789.012345678	457247366712391403337905807.00045724733
-123456789012345678901234567890.12345678	4572473667123914033379058070.0045724733
-1234567890123456789012345678901.2345678	45724736671239140333790580700.045724733
-12345678901234567890123456789012.345678	457247366712391403337905807000.45724733
-123456789012345678901234567890123.45678	4572473667123914033379058070004.5724733
-1234567890123456789012345678901234.5678	45724736671239140333790580700045.724733
-12345678901234567890123456789012345.678	457247366712391403337905807000457.24733
-99999999999999999999999999999999999	3703703703703703703703703703703703.6667
-123456789012345678901234567890123456.78	4572473667123914033379058070004572.4733
-999999999999999999999999999999999999	37037037037037037037037037037037037
-12345678901234567890123456789012345678	457247366712391403337905807000457247.33
-12345678901234567890123456789012345678	457247366712391403337905807000457247.33
-12345678901234567890123456789012345678	457247366712391403337905807000457247.33
-12345678901234567890123456789012345678	457247366712391403337905807000457247.33
-12345678901234567890123456789012345678	457247366712391403337905807000457247.33
-99999999999999999999999999999999999999	3703703703703703703703703703703703703.7
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e b/sql/hive/src/test/resources/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e
deleted file mode 100644
index c1e0db0174c63dc9004cc798b011fa82517ae8df..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e
+++ /dev/null
@@ -1,75 +0,0 @@
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
-NULL	NULL
--99999999999999999999999999999999999999	NULL
--999999999999999999999999999999999999	NULL
--99999999999999999999999999999999999	NULL
--0.0000000000000000000000000000000000001	NULL
-0	0
-0.0000000000000000000000000000000000001	NULL
-0.123456789012345	0.015241578753238669120562399025
-0.1234567890123456789012345678901234578	NULL
-1.234567890123456	1.524157875323881726870921383936
-1.2345678901234567890123456789012345678	NULL
-12.34567890123456	152.4157875323881726870921383936
-12.345678901234567890123456789012345678	NULL
-123.4567890123456	15241.57875323881726870921383936
-123.45678901234567890123456789012345678	NULL
-1234.567890123456	1524157.875323881726870921383936
-1234.5678901234567890123456789012345678	NULL
-12345.67890123456	152415787.5323881726870921383936
-12345.678901234567890123456789012345678	NULL
-123456.7890123456	15241578753.23881726870921383936
-123456.78901234567890123456789012345678	NULL
-1234567.890123456	1524157875323.881726870921383936
-1234567.8901234567890123456789012345678	NULL
-12345678.90123456	152415787532388.1726870921383936
-12345678.901234567890123456789012345678	NULL
-123456789.0123456	15241578753238817.26870921383936
-123456789.01234567890123456789012345678	NULL
-1234567890.123456	1524157875323881726.870921383936
-1234567890.1234567890123456789012345678	NULL
-12345678901.23456	152415787532388172687.0921383936
-12345678901.234567890123456789012345678	NULL
-123456789012.3456	15241578753238817268709.21383936
-123456789012.34567890123456789012345678	NULL
-1234567890123.456	1524157875323881726870921.383936
-1234567890123.4567890123456789012345678	NULL
-12345678901234.56	152415787532388172687092138.3936
-12345678901234.567890123456789012345678	NULL
-123456789012345.6	15241578753238817268709213839.36
-123456789012345.67890123456789012345678	NULL
-1234567890123456.7890123456789012345678	NULL
-12345678901234567.890123456789012345678	NULL
-123456789012345678.90123456789012345678	NULL
-1234567890123456789.0123456789012345678	NULL
-12345678901234567890.123456789012345678	NULL
-123456789012345678901.23456789012345678	NULL
-1234567890123456789012.3456789012345678	NULL
-12345678901234567890123.456789012345678	NULL
-123456789012345678901234.56789012345678	NULL
-1234567890123456789012345.6789012345678	NULL
-12345678901234567890123456.789012345678	NULL
-123456789012345678901234567.89012345678	NULL
-1234567890123456789012345678.9012345678	NULL
-12345678901234567890123456789.012345678	NULL
-123456789012345678901234567890.12345678	NULL
-1234567890123456789012345678901.2345678	NULL
-12345678901234567890123456789012.345678	NULL
-123456789012345678901234567890123.45678	NULL
-1234567890123456789012345678901234.5678	NULL
-12345678901234567890123456789012345.678	NULL
-99999999999999999999999999999999999	NULL
-123456789012345678901234567890123456.78	NULL
-999999999999999999999999999999999999	NULL
-12345678901234567890123456789012345678	NULL
-12345678901234567890123456789012345678	NULL
-12345678901234567890123456789012345678	NULL
-12345678901234567890123456789012345678	NULL
-12345678901234567890123456789012345678	NULL
-99999999999999999999999999999999999999	NULL
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab b/sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab
deleted file mode 100644
index 81af0e4cd3ab8bded4226d0c55fc4595d751f317..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab
+++ /dev/null
@@ -1 +0,0 @@
-NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c b/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c
+++ b/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be b/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be
index ded23df148827aa1943204c5615d68504c674f07..3aa0e30600f3caf649e1bdb94d35b6b9b92ac674 100644
--- a/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be
+++ b/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be
@@ -1 +1 @@
-ds=__HIVE_DEFAULT_PARTITION__
\ No newline at end of file
+ds=__HIVE_DEFAULT_PARTITION__
diff --git a/sql/hive/src/test/resources/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 b/sql/hive/src/test/resources/golden/delimiter-1-121ff21e6931a82235de8301118cbed8
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8
rename to sql/hive/src/test/resources/golden/delimiter-1-121ff21e6931a82235de8301118cbed8
diff --git a/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 b/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069
index f438072c76b5f9d098f8fdd21ef760674600f3f3..424a2fee069876320119350dd58d1397e5e4119a 100644
--- a/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069
+++ b/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069
@@ -1,3 +1,3 @@
 35	40
 48	32
-100100	40
\ No newline at end of file
+100100	40
diff --git a/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 b/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7
index f438072c76b5f9d098f8fdd21ef760674600f3f3..424a2fee069876320119350dd58d1397e5e4119a 100644
--- a/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7
+++ b/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7
@@ -1,3 +1,3 @@
 35	40
 48	32
-100100	40
\ No newline at end of file
+100100	40
diff --git a/sql/hive/src/test/resources/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 b/sql/hive/src/test/resources/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904
deleted file mode 100644
index d980efc81b9470b4eb1555ce6bbf1986c6675784..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904
+++ /dev/null
@@ -1,6 +0,0 @@
-col1                	int                 	col1 one line comment
-col2                	string              	col2                
-                    	                    	two lines comment
-col3                	string              	col3                
-                    	                    	three lines
-                    	                    	comment
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 b/sql/hive/src/test/resources/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94
deleted file mode 100644
index 01b9151074b22f86af0e8846b30e429f76be4037..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-col1                	int                 	col1 one line comment
-col2                	string              	col2                
-                    	                    	two lines comment
-col3                	string              	col3                
-                    	                    	three lines
-                    	                    	comment
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 14 11:42:35 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/test_table	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	comment             	table comment\ntwo lines
-	transient_lastDdlTime	1389728555          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 b/sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a b/sql/hive/src/test/resources/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a
deleted file mode 100644
index c56a79e4f322e2400841673d09ea5e6bd6e3c7ea..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a
+++ /dev/null
@@ -1 +0,0 @@
-{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsondb1","comment":"Test database","database":"jsondb1"}	 	 	 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 b/sql/hive/src/test/resources/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304
deleted file mode 100644
index aa08c38c68d1d1a4ef97fe9c6199bccfad21612a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304
+++ /dev/null
@@ -1 +0,0 @@
-{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsondb1","params":{"id":"jsondb1"},"comment":"Test database","database":"jsondb1"}	 	 	 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 b/sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20
deleted file mode 100644
index 513aeaab1dc6639aef5630dd3d0dce4e361ceabc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20
+++ /dev/null
@@ -1 +0,0 @@
-{"databases":["default","jsondb1"]}
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c b/sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c
deleted file mode 100644
index 606069d6291b43bd9a609eeb1871f0ef0e8bcbcd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c
+++ /dev/null
@@ -1 +0,0 @@
-{"databases":["jsondb1"]}
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a b/sql/hive/src/test/resources/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a
deleted file mode 100644
index bea7c01440c466a4112bded698e4c444e190dd4f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a
+++ /dev/null
@@ -1 +0,0 @@
-{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsondb1.db","database":"jsondb1"}	 	 	 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 b/sql/hive/src/test/resources/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304
deleted file mode 100644
index bea7c01440c466a4112bded698e4c444e190dd4f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304
+++ /dev/null
@@ -1 +0,0 @@
-{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsondb1.db","database":"jsondb1"}	 	 	 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826
index 381821184d69310c523948869f77ee8a9a9f26b6..1ed0de6860c084ba8e9e387a9a23c6a48dda7899 100644
--- a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826
+++ b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826
@@ -1,19 +1,19 @@
 # col_name            	data_type           	comment             
 	 	 
-key                 	int                 	None                
+key                 	int                 	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-value               	string              	None                
+value               	string              	                    
 	 	 
 # Detailed Partition Information	 	 
 Partition Value:    	[val_86]            	 
 Database:           	default             	 
 Table:              	view_partitioned    	 
-CreateTime:         	Fri Feb 07 15:09:16 PST 2014	 
+CreateTime:         	Tue Oct 21 01:26:15 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Location:           	null                	 
 Partition Parameters:	 	 
-	transient_lastDdlTime	1391814556          
+	transient_lastDdlTime	1413879975          
diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826
deleted file mode 100644
index 7b51873776ad8b9e40052598dcc08eee2b08be67..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826
+++ /dev/null
@@ -1 +0,0 @@
-{"columns":[{"name":"key","type":"int"}]}	 	 
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf b/sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf b/sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 b/sql/hive/src/test/resources/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264
deleted file mode 100644
index 35fac1b6f2579d1be70374916e41b643731cb599..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264
+++ /dev/null
@@ -1,12 +0,0 @@
-col_name                          data_type     comment	 	 
-	 	 
-col1                              int           col1 one line comment	 	 
-col2_abcdefghiklmnopqrstuvxyz     string        col2	 	 
-                                                two lines comment	 	 
-col3                              string        col3	 	 
-                                                three lines	 	 
-                                                comment	 	 
-col4                              string        col4 very long comment that is	 	 
-                                                greater than 80 chars and is	 	 
-                                                likely to spill into multiple	 	 
-                                                lines	 	 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 b/sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 b/sql/hive/src/test/resources/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264
deleted file mode 100644
index b57f8955ca397c8cebfdbc9584e2d15eb263c2a8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264
+++ /dev/null
@@ -1,24 +0,0 @@
-col_name                          data_type     comment	 	 
-	 	 
-col1                              int           col1 one	 	 
-                                                line	 	 
-                                                comment	 	 
-col2_abcdefghiklmnopqrstuvxyz     string        col2	 	 
-                                                two lines	 	 
-                                                comment	 	 
-col3                              string        col3	 	 
-                                                three	 	 
-                                                lines	 	 
-                                                comment	 	 
-col4                              string        col4 very	 	 
-                                                long	 	 
-                                                comment	 	 
-                                                that is	 	 
-                                                greater	 	 
-                                                than 80	 	 
-                                                chars and	 	 
-                                                is likely	 	 
-                                                to spill	 	 
-                                                into	 	 
-                                                multiple	 	 
-                                                lines	 	 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 b/sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 b/sql/hive/src/test/resources/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22
deleted file mode 100644
index 3b7fe3c1330890a2aeda0fb4404da8a013f173df..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22
+++ /dev/null
@@ -1,37 +0,0 @@
-col_name           data_type     comment	 	 
-	 	 
-col1               int           col1 one line comment	 	 
-col2               string        col2	 	 
-                                 two lines comment	 	 
-col3               string        col3	 	 
-                                 three lines	 	 
-                                 comment	 	 
-col4               string        col4 very long comment that	 	 
-                                 is greater than 80 chars	 	 
-                                 and is likely to spill into	 	 
-                                 multiple lines	 	 
-col5               string        col5 very long multi-line	 	 
-                                 comment where each line is	 	 
-                                 very long by itself and is	 	 
-                                 likely to spill	 	 
-                                 into multiple lines.  Lorem	 	 
-                                 ipsum dolor sit amet,	 	 
-                                 consectetur adipiscing	 	 
-                                 elit. Proin in dolor nisl,	 	 
-                                 sodales	 	 
-                                 adipiscing tortor. Integer	 	 
-                                 venenatis	 	 
-col6               string        This comment has a very	 	 
-                                 long single word ABCDEFGHIJ	 	 
-                                 KLMNOPQRSTUVXYZabcdefghijkl	 	 
-                                 mnopqrstuvzxyz123 which	 	 
-                                 will not fit in a line by	 	 
-                                 itself for small column	 	 
-                                 widths.	 	 
-col7_nocomment     string        None	 	 
-ds                 string        None	 	 
-	 	 
-# Partition Information	 	 
-col_name           data_type     comment	 	 
-	 	 
-ds                 string        None	 	 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 b/sql/hive/src/test/resources/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861
deleted file mode 100644
index 49175da27357f6bf5293be350685f38c8b04ce9b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861
+++ /dev/null
@@ -1,23 +0,0 @@
-col_name           data_type     comment	 	 
-	 	 
-col1               int           col1 one line comment	 	 
-col2               string        col2	 	 
-                                 two lines comment	 	 
-col3               string        col3	 	 
-                                 three lines	 	 
-                                 comment	 	 
-col4               string        col4 very long comment that is greater than 80	 	 
-                                 chars and is likely to spill into multiple	 	 
-                                 lines	 	 
-col5               string        col5 very long multi-line comment where each	 	 
-                                 line is very long by itself and is likely to	 	 
-                                 spill	 	 
-                                 into multiple lines.  Lorem ipsum dolor sit	 	 
-                                 amet, consectetur adipiscing elit. Proin in	 	 
-                                 dolor nisl, sodales	 	 
-                                 adipiscing tortor. Integer venenatis	 	 
-col6               string        This comment has a very long single word ABCDEF	 	 
-                                 GHIJKLMNOPQRSTUVXYZabcdefghijklmnopqrstuvzxyz12	 	 
-                                 3 which will not fit in a line by itself for	 	 
-                                 small column widths.	 	 
-col7_nocomment     string        None	 	 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 b/sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 b/sql/hive/src/test/resources/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255
deleted file mode 100644
index c3e77e079a9d49c52d0d3c767e96a48c60cf07ea..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255
+++ /dev/null
@@ -1,14 +0,0 @@
-col_name           data_type     comment	 	 
-	 	 
-col1               int           col1 one line comment	 	 
-col2               string        col2	 	 
-                                 two lines comment	 	 
-col3               string        col3	 	 
-                                 three lines	 	 
-                                 comment	 	 
-col4               string        col4 very long comment that is greater than 80 chars and is likely to spill into multiple lines	 	 
-col5               string        col5 very long multi-line comment where each line is very long by itself and is likely to spill	 	 
-                                 into multiple lines.  Lorem ipsum dolor sit amet, consectetur adipiscing elit. Proin in dolor nisl, sodales	 	 
-                                 adipiscing tortor. Integer venenatis	 	 
-col6               string        This comment has a very long single word ABCDEFGHIJKLMNOPQRSTUVXYZabcdefghijklmnopqrstuvzxyz123 which will not fit in a line by itself for small column widths.	 	 
-col7_nocomment     string        None	 	 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 b/sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 b/sql/hive/src/test/resources/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255
deleted file mode 100644
index 452f75a11fdd0076b333b82389557cabf9e45b1e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255
+++ /dev/null
@@ -1,44 +0,0 @@
-col_name           data_type     comment	 	 
-	 	 
-col1               int           col1 one line	 	 
-                                 comment	 	 
-col2               string        col2	 	 
-                                 two lines comment	 	 
-col3               string        col3	 	 
-                                 three lines	 	 
-                                 comment	 	 
-col4               string        col4 very long	 	 
-                                 comment that is	 	 
-                                 greater than 80	 	 
-                                 chars and is	 	 
-                                 likely to spill	 	 
-                                 into multiple	 	 
-                                 lines	 	 
-col5               string        col5 very long	 	 
-                                 multi-line	 	 
-                                 comment where	 	 
-                                 each line is very	 	 
-                                 long by itself	 	 
-                                 and is likely to	 	 
-                                 spill	 	 
-                                 into multiple	 	 
-                                 lines.  Lorem	 	 
-                                 ipsum dolor sit	 	 
-                                 amet, consectetur	 	 
-                                 adipiscing elit.	 	 
-                                 Proin in dolor	 	 
-                                 nisl, sodales	 	 
-                                 adipiscing	 	 
-                                 tortor. Integer	 	 
-                                 venenatis	 	 
-col6               string        This comment has	 	 
-                                 a very long	 	 
-                                 single word ABCDE	 	 
-                                 FGHIJKLMNOPQRSTUV	 	 
-                                 XYZabcdefghijklmn	 	 
-                                 opqrstuvzxyz123	 	 
-                                 which will not	 	 
-                                 fit in a line by	 	 
-                                 itself for small	 	 
-                                 column widths.	 	 
-col7_nocomment     string        None	 	 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 b/sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 b/sql/hive/src/test/resources/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255
deleted file mode 100644
index ee5a10c85057a350026e8aca7490be5f41df7655..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255
+++ /dev/null
@@ -1,31 +0,0 @@
-col_name           data_type     comment	 	 
-	 	 
-col1               int           col1 one line comment	 	 
-col2               string        col2	 	 
-                                 two lines comment	 	 
-col3               string        col3	 	 
-                                 three lines	 	 
-                                 comment	 	 
-col4               string        col4 very long comment that	 	 
-                                 is greater than 80 chars	 	 
-                                 and is likely to spill into	 	 
-                                 multiple lines	 	 
-col5               string        col5 very long multi-line	 	 
-                                 comment where each line is	 	 
-                                 very long by itself and is	 	 
-                                 likely to spill	 	 
-                                 into multiple lines.  Lorem	 	 
-                                 ipsum dolor sit amet,	 	 
-                                 consectetur adipiscing	 	 
-                                 elit. Proin in dolor nisl,	 	 
-                                 sodales	 	 
-                                 adipiscing tortor. Integer	 	 
-                                 venenatis	 	 
-col6               string        This comment has a very	 	 
-                                 long single word ABCDEFGHIJ	 	 
-                                 KLMNOPQRSTUVXYZabcdefghijkl	 	 
-                                 mnopqrstuvzxyz123 which	 	 
-                                 will not fit in a line by	 	 
-                                 itself for small column	 	 
-                                 widths.	 	 
-col7_nocomment     string        None	 	 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 b/sql/hive/src/test/resources/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5
deleted file mode 100644
index 4184ce21dc0792ebda477a5dbc58657327b8ad5d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5
+++ /dev/null
@@ -1,12 +0,0 @@
-key1                	int                 	None                
-value1              	string              	None                
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-Detailed Table Information	Table(tableName:t1, dbName:db1, owner:marmbrus, createTime:1389728588, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key1, type:int, comment:null), FieldSchema(name:value1, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728588}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f b/sql/hive/src/test/resources/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f
deleted file mode 100644
index c94d6dcb9004251fdf285fc16a63a142fd96feee..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f
+++ /dev/null
@@ -1,33 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key1                	int                 	None                
-value1              	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	db1                 	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 14 11:43:08 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1389728588          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 b/sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4
deleted file mode 100644
index 0dea48c260ab2c68eb843cb0fce0f0f06c0f6023..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4
+++ /dev/null
@@ -1 +0,0 @@
-key1                	int                 	from deserializer   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 b/sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758
deleted file mode 100644
index 0dea48c260ab2c68eb843cb0fce0f0f06c0f6023..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758
+++ /dev/null
@@ -1 +0,0 @@
-key1                	int                 	from deserializer   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 b/sql/hive/src/test/resources/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9
deleted file mode 100644
index f3d242157dd98a742c2c6518c5a85e141bdc0d10..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9
+++ /dev/null
@@ -1,3 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key1                	int                 	from deserializer   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 b/sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4
deleted file mode 100644
index 0dea48c260ab2c68eb843cb0fce0f0f06c0f6023..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4
+++ /dev/null
@@ -1 +0,0 @@
-key1                	int                 	from deserializer   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 b/sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7
deleted file mode 100644
index 0dea48c260ab2c68eb843cb0fce0f0f06c0f6023..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7
+++ /dev/null
@@ -1 +0,0 @@
-key1                	int                 	from deserializer   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 b/sql/hive/src/test/resources/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58
deleted file mode 100644
index f3d242157dd98a742c2c6518c5a85e141bdc0d10..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58
+++ /dev/null
@@ -1,3 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key1                	int                 	from deserializer   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 b/sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4
deleted file mode 100644
index 0dea48c260ab2c68eb843cb0fce0f0f06c0f6023..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4
+++ /dev/null
@@ -1 +0,0 @@
-key1                	int                 	from deserializer   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa b/sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa
deleted file mode 100644
index 0dea48c260ab2c68eb843cb0fce0f0f06c0f6023..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa
+++ /dev/null
@@ -1 +0,0 @@
-key1                	int                 	from deserializer   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a b/sql/hive/src/test/resources/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a
deleted file mode 100644
index f3d242157dd98a742c2c6518c5a85e141bdc0d10..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a
+++ /dev/null
@@ -1,3 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key1                	int                 	from deserializer   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 b/sql/hive/src/test/resources/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3
deleted file mode 100644
index 4f76eaca6cd8be0376ee24bb7b67c22971d8994c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3
+++ /dev/null
@@ -1,10 +0,0 @@
-key1                	int                 	None                
-value1              	string              	None                
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-part                	string              	None                
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f b/sql/hive/src/test/resources/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f
deleted file mode 100644
index aa25ca5a29dd39a8d33787fd62b833659a447a07..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f
+++ /dev/null
@@ -1,12 +0,0 @@
-key1                	int                 	None                
-value1              	string              	None                
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-Detailed Partition Information	Partition(values:[4, 5], dbName:db1, tableName:t1, createTime:1389728588, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key1, type:int, comment:null), FieldSchema(name:value1, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1/ds=4/part=5, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1389728588})	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 b/sql/hive/src/test/resources/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49
deleted file mode 100644
index 311870f6ad6b038842ec181400351edfba996970..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key1                	int                 	None                
-value1              	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[4, 5]              	 
-Database:           	db1                 	 
-Table:              	t1                  	 
-CreateTime:         	Tue Jan 14 11:43:08 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1/ds=4/part=5	 
-Partition Parameters:	 	 
-	transient_lastDdlTime	1389728588          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-24-ee226b42db35b219702319858e925468 b/sql/hive/src/test/resources/golden/describe_syntax-24-ee226b42db35b219702319858e925468
deleted file mode 100644
index 4f76eaca6cd8be0376ee24bb7b67c22971d8994c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-24-ee226b42db35b219702319858e925468
+++ /dev/null
@@ -1,10 +0,0 @@
-key1                	int                 	None                
-value1              	string              	None                
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-part                	string              	None                
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e b/sql/hive/src/test/resources/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e
deleted file mode 100644
index aa25ca5a29dd39a8d33787fd62b833659a447a07..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e
+++ /dev/null
@@ -1,12 +0,0 @@
-key1                	int                 	None                
-value1              	string              	None                
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-Detailed Partition Information	Partition(values:[4, 5], dbName:db1, tableName:t1, createTime:1389728588, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key1, type:int, comment:null), FieldSchema(name:value1, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1/ds=4/part=5, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1389728588})	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec b/sql/hive/src/test/resources/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec
deleted file mode 100644
index 311870f6ad6b038842ec181400351edfba996970..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key1                	int                 	None                
-value1              	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[4, 5]              	 
-Database:           	db1                 	 
-Table:              	t1                  	 
-CreateTime:         	Tue Jan 14 11:43:08 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1/ds=4/part=5	 
-Partition Parameters:	 	 
-	transient_lastDdlTime	1389728588          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 b/sql/hive/src/test/resources/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85
deleted file mode 100644
index 4f76eaca6cd8be0376ee24bb7b67c22971d8994c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85
+++ /dev/null
@@ -1,10 +0,0 @@
-key1                	int                 	None                
-value1              	string              	None                
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-part                	string              	None                
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d b/sql/hive/src/test/resources/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d
deleted file mode 100644
index 4184ce21dc0792ebda477a5dbc58657327b8ad5d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d
+++ /dev/null
@@ -1,12 +0,0 @@
-key1                	int                 	None                
-value1              	string              	None                
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-Detailed Table Information	Table(tableName:t1, dbName:db1, owner:marmbrus, createTime:1389728588, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key1, type:int, comment:null), FieldSchema(name:value1, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728588}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 b/sql/hive/src/test/resources/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4
deleted file mode 100644
index c94d6dcb9004251fdf285fc16a63a142fd96feee..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4
+++ /dev/null
@@ -1,33 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key1                	int                 	None                
-value1              	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	db1                 	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 14 11:43:08 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1389728588          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b b/sql/hive/src/test/resources/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b
deleted file mode 100644
index 4f76eaca6cd8be0376ee24bb7b67c22971d8994c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b
+++ /dev/null
@@ -1,10 +0,0 @@
-key1                	int                 	None                
-value1              	string              	None                
-ds                  	string              	None                
-part                	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-part                	string              	None                
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c
deleted file mode 100644
index f8bc404bf73089aa178a09c4e03bd517b6a0ce4a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c
+++ /dev/null
@@ -1 +0,0 @@
-{"tables":["jsontable","src","srcpart"]}
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a b/sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a
deleted file mode 100644
index 5895645dbbb5026c03660cbb71035a5aa4852cd9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a
+++ /dev/null
@@ -1 +0,0 @@
-{"tables":["jsontable"]}
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 b/sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721
deleted file mode 100644
index 353bf2df92f18866c28099465ab544337b0ce2c0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721
+++ /dev/null
@@ -1 +0,0 @@
-{"tables":[]}
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 b/sql/hive/src/test/resources/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430
deleted file mode 100644
index 96c1178ae6eab7080c94ea30f08ef1c41a54e15b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430
+++ /dev/null
@@ -1 +0,0 @@
-{"columns":[{"name":"key","type":"int"},{"name":"value","type":"string"}]}	 	 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 b/sql/hive/src/test/resources/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30
deleted file mode 100644
index 4cf10d1d762b04abad76a0b2321971688023fdab..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30
+++ /dev/null
@@ -1 +0,0 @@
-{"columns":[{"name":"key","type":"int"},{"name":"value","type":"string"}],"tableInfo":{"owner":"marmbrus","parameters":{"id":"jsontable","last_modified_by":"marmbrus","last_modified_time":"1389728616","transient_lastDdlTime":"1389728616","comment":"json table"},"createTime":1389728615,"dbName":"default","tableName":"jsontable","privileges":null,"tableType":"MANAGED_TABLE","sd":{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsontable","parameters":{},"inputFormat":"org.apache.hadoop.mapred.TextInputFormat","outputFormat":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","compressed":false,"cols":[{"name":"key","type":"int","comment":null,"setName":true,"setType":true,"setComment":false},{"name":"value","type":"string","comment":null,"setName":true,"setType":true,"setComment":false}],"serdeInfo":{"name":null,"parameters":{"serialization.format":"1"},"serializationLib":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","setName":false,"parametersSize":1,"setParameters":true,"setSerializationLib":true},"colsSize":2,"skewedInfo":{"skewedColNames":[],"skewedColValues":[],"skewedColValueLocationMaps":{},"skewedColNamesSize":0,"skewedColNamesIterator":[],"setSkewedColNames":true,"skewedColValuesSize":0,"skewedColValuesIterator":[],"setSkewedColValues":true,"skewedColValueLocationMapsSize":0,"setSkewedColValueLocationMaps":true},"bucketCols":[],"numBuckets":-1,"sortCols":[],"storedAsSubDirectories":false,"setSkewedInfo":true,"parametersSize":0,"setParameters":true,"colsIterator":[{"name":"key","type":"int","comment":null,"setName":true,"setType":true,"setComment":false},{"name":"value","type":"string","comment":null,"setName":true,"setType":true,"setComment":false}],"setCols":true,"setLocation":true,"setInputFormat":true,"setOutputFormat":true,"setCompressed":true,"setNumBuckets":true,"setSerdeInfo":true,"bucketColsSize":0,"bucketColsIterator":[],"setBucketCols":true,"sortColsSize":0,"sortColsIterator":[],"setSortCols":true,"setStoredAsSubDirectories":true},"partitionKeys":[],"viewOriginalText":null,"lastAccessTime":0,"retention":0,"viewExpandedText":null,"partitionKeysSize":0,"setTableType":true,"setTableName":true,"setDbName":true,"setOwner":true,"setCreateTime":true,"setLastAccessTime":true,"setRetention":true,"setSd":true,"partitionKeysIterator":[],"setPartitionKeys":true,"parametersSize":5,"setParameters":true,"setViewOriginalText":false,"setViewExpandedText":false,"setPrivileges":false}}	 	 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 b/sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b b/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b
+++ b/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c b/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c
+++ b/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 b/sql/hive/src/test/resources/golden/disable_file_format_check-2-2d27f92dfced693fa3a68ecce5e2e838
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64
rename to sql/hive/src/test/resources/golden/disable_file_format_check-2-2d27f92dfced693fa3a68ecce5e2e838
diff --git a/sql/hive/src/test/resources/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 b/sql/hive/src/test/resources/golden/disable_file_format_check-4-a2150709a6ff73326bdf4865dd124a23
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67
rename to sql/hive/src/test/resources/golden/disable_file_format_check-4-a2150709a6ff73326bdf4865dd124a23
diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-a071dedef216e84d1cb2f0de6d34fd1a
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-a071dedef216e84d1cb2f0de6d34fd1a
diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b
deleted file mode 100644
index 7ae602958428e9f47d16bf703e46aeea97a938a3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b
+++ /dev/null
@@ -1 +0,0 @@
-238	val_238
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-ce3797dc14a603cba2a5e58c8612de5b
similarity index 100%
rename from sql/hive/src/test/resources/golden/read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b
rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-ce3797dc14a603cba2a5e58c8612de5b
diff --git a/sql/hive/src/test/resources/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-f5340880d2be7b0643eb995673e89d11
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8
rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-f5340880d2be7b0643eb995673e89d11
diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c
deleted file mode 100644
index ca21e093aa698b7b1279c8fdb5ec483fb71466ee..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c
+++ /dev/null
@@ -1 +0,0 @@
-1	{"a1":"b1"}	foo1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-714ab8c97f4d8993680b91e1ed8f3782
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0
rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-714ab8c97f4d8993680b91e1ed8f3782
diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-34064fd15c28dba55865cb8f3c5ba68c b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-34064fd15c28dba55865cb8f3c5ba68c
new file mode 100644
index 0000000000000000000000000000000000000000..573c4b56de599c414c45ff0d96a9ea8ee5c9006d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-34064fd15c28dba55865cb8f3c5ba68c
@@ -0,0 +1 @@
+1	{"a1":"b1"}	foo1
diff --git a/sql/hive/src/test/resources/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-6-f40a07d7654573e1a8517770eb8529e7
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711
rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-6-f40a07d7654573e1a8517770eb8529e7
diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/distinct_stats-0-418ec894d08c33fd712eb358f579b7a0
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/distinct_stats-0-418ec894d08c33fd712eb358f579b7a0
diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/distinct_stats-1-10987e425ba8ba8d9c01538f16eab970
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/distinct_stats-1-10987e425ba8ba8d9c01538f16eab970
diff --git a/sql/hive/src/test/resources/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 b/sql/hive/src/test/resources/golden/distinct_stats-2-a2d8f812612283b20ec3f1e92a263440
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409
rename to sql/hive/src/test/resources/golden/distinct_stats-2-a2d8f812612283b20ec3f1e92a263440
diff --git a/sql/hive/src/test/resources/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 b/sql/hive/src/test/resources/golden/distinct_stats-3-a4397664f1f109ef0aa5ff36961b25b
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7
rename to sql/hive/src/test/resources/golden/distinct_stats-3-a4397664f1f109ef0aa5ff36961b25b
diff --git a/sql/hive/src/test/resources/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 b/sql/hive/src/test/resources/golden/distinct_stats-4-e540680af4a857404a0cb7cabc1bcf31
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805
rename to sql/hive/src/test/resources/golden/distinct_stats-4-e540680af4a857404a0cb7cabc1bcf31
diff --git a/sql/hive/src/test/resources/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 b/sql/hive/src/test/resources/golden/distinct_stats-5-32e9736bf27c1d2e4399a8125e14befc
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9
rename to sql/hive/src/test/resources/golden/distinct_stats-5-32e9736bf27c1d2e4399a8125e14befc
diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc b/sql/hive/src/test/resources/golden/distinct_stats-6-297a53801744e6c4786e315e32c6189a
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc
rename to sql/hive/src/test/resources/golden/distinct_stats-6-297a53801744e6c4786e315e32c6189a
diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 b/sql/hive/src/test/resources/golden/distinct_stats-7-92b9ef922e6b63a9de3ebcc23ee2d02d
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18
rename to sql/hive/src/test/resources/golden/distinct_stats-7-92b9ef922e6b63a9de3ebcc23ee2d02d
diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 b/sql/hive/src/test/resources/golden/distinct_stats-8-ca16024e6f5399b1d035f5b9fd665163
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96
rename to sql/hive/src/test/resources/golden/distinct_stats-8-ca16024e6f5399b1d035f5b9fd665163
diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/distinct_stats-9-4b2407991ccf180e0eb38bf3d2ef2ec8
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/distinct_stats-9-4b2407991ccf180e0eb38bf3d2ef2ec8
diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-2-fb7b53f61989f4f645dac4a8f017d6ee
similarity index 100%
rename from sql/hive/src/test/resources/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16
rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-2-fb7b53f61989f4f645dac4a8f017d6ee
diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-3-46fe5bb027667f528d7179b239e3427f
similarity index 100%
rename from sql/hive/src/test/resources/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e
rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-3-46fe5bb027667f528d7179b239e3427f
diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-4-26dcd2b2f263b5b417430efcf354663a
similarity index 100%
rename from sql/hive/src/test/resources/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5
rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-4-26dcd2b2f263b5b417430efcf354663a
diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-5-2a1bd5ed3955825a9dbb76769f7fe4ea
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-5-2a1bd5ed3955825a9dbb76769f7fe4ea
diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-6-7a9e67189d3d4151f23b12c22bde06b5
similarity index 100%
rename from sql/hive/src/test/resources/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32
rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-6-7a9e67189d3d4151f23b12c22bde06b5
diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-7-16c31455a193e1cb06a2ede4e9f5d5dd
similarity index 100%
rename from sql/hive/src/test/resources/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b
rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-7-16c31455a193e1cb06a2ede4e9f5d5dd
diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-8-2a1bd5ed3955825a9dbb76769f7fe4ea
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-8-2a1bd5ed3955825a9dbb76769f7fe4ea
diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-9-40110efef10f6f7b873dcd1d53463101
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-9-40110efef10f6f7b873dcd1d53463101
diff --git a/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48
+++ b/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48
+++ b/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-0-a99af48bbcbaba062e9bc387ae2b4975
similarity index 100%
rename from sql/hive/src/test/resources/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44
rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-0-a99af48bbcbaba062e9bc387ae2b4975
diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-1-e67a0f7ff61a97b2c49386890ea88c54
similarity index 100%
rename from sql/hive/src/test/resources/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f
rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-1-e67a0f7ff61a97b2c49386890ea88c54
diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-2-19915227905aab376d918b3cada85c25
similarity index 100%
rename from sql/hive/src/test/resources/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539
rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-2-19915227905aab376d918b3cada85c25
diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-3-e32b952789a115ff02201dfa618d92b2
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-3-e32b952789a115ff02201dfa618d92b2
diff --git a/sql/hive/src/test/resources/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-4-58aff7fa05fba3a7549629a17e285036
similarity index 100%
rename from sql/hive/src/test/resources/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5
rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-4-58aff7fa05fba3a7549629a17e285036
diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-5-e32b952789a115ff02201dfa618d92b2
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-5-e32b952789a115ff02201dfa618d92b2
diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-6-4642f8a18bf7409849f2e91d7a05f352
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-6-4642f8a18bf7409849f2e91d7a05f352
diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 b/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760
index c9d701778f9abe0f0308120d51bdf2f42081b308..2895d472ca5d96207cf189fd562564ade767a33e 100644
--- a/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760
+++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760
@@ -1 +1 @@
-b=2/c=2
\ No newline at end of file
+b=2/c=2
diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 b/sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760
index 53e09b6e34202dc7ebfbc9bd03d1430d7b34e984..0f9c2f1d90639e480400256b387c3c4ac1104852 100644
--- a/sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760
+++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760
@@ -1,3 +1,3 @@
 b=1/c=1
 b=1/c=2
-b=2/c=2
\ No newline at end of file
+b=2/c=2
diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 b/sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0
index 31b543e8b4122bb3b5fcd6b3073f3864be391435..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 100644
--- a/sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0
+++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0
@@ -1,12 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ALTERTABLE_DROPPARTS mp (TOK_PARTSPEC (TOK_PARTVAL b = '1')))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Drop Table Operator:
-        Drop Table
-          table: mp
-
diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 b/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760
index c9d701778f9abe0f0308120d51bdf2f42081b308..2895d472ca5d96207cf189fd562564ade767a33e 100644
--- a/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760
+++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760
@@ -1 +1 @@
-b=2/c=2
\ No newline at end of file
+b=2/c=2
diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 b/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1
+++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 b/sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6
index d1e9fd1cd0a21e2ea39293e2905ea75849d709ad..30eb53bdc8e70fb2f79b44707a3160629ad6cb8a 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6
@@ -1,10 +1,10 @@
-a                   	string              	None                
-b                   	int                 	None                
-c                   	string              	None                
-d                   	string              	None                
+a                   	string              	                    
+b                   	int                 	                    
+c                   	string              	                    
+d                   	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-c                   	string              	None                
-d                   	string              	None                
\ No newline at end of file
+c                   	string              	                    
+d                   	string              	                    
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7
index 3377ef5cf498f35a1eefe1faaafd092eaf15bb74..5e6d96c4e60aa336c91af1c2ad7a35a5f134da10 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7
@@ -6,4 +6,4 @@ c=India/d=3
 c=Russia/d=3
 c=US/d=1
 c=US/d=2
-c=Uganda/d=2
\ No newline at end of file
+c=Uganda/d=2
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7
index 40e71fb79ad0f90273bebcf9c5232a1b681cf0e4..e91541d1527d70d52f1eff6bfdb24dcf4c5286da 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7
@@ -5,4 +5,4 @@ c=Greece/d=2
 c=India/d=3
 c=Russia/d=3
 c=US/d=2
-c=Uganda/d=2
\ No newline at end of file
+c=Uganda/d=2
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7
index c03d86a551c296fc5a2ab067634096da5bf879de..316e63e21e7b11ca7f17281a9a3a1a8b560d6bae 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7
@@ -3,4 +3,4 @@ c=France/d=4
 c=Germany/d=2
 c=Greece/d=2
 c=India/d=3
-c=Russia/d=3
\ No newline at end of file
+c=Russia/d=3
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7
index 133c0256f898f1cc630278ca65feeafceae0dc1b..231c59f36530785bab455bb640c312bb825daad7 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7
@@ -2,4 +2,4 @@ c=Canada/d=3
 c=France/d=4
 c=Germany/d=2
 c=Greece/d=2
-c=India/d=3
\ No newline at end of file
+c=India/d=3
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7
index 98e52eedc2b0318126efd531578532d9b85fba3f..02a7003ca12de477dd02fe98b43b5beb05512f38 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7
@@ -1,3 +1,3 @@
 c=Canada/d=3
 c=France/d=4
-c=Germany/d=2
\ No newline at end of file
+c=Germany/d=2
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7
index 687f7dd7e13a073314210d1ea243891bdb8b26fa..2ffea3b50a9b4211fa8110f989822d9ac3b58ee9 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7
@@ -1 +1 @@
-c=France/d=4
\ No newline at end of file
+c=France/d=4
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 b/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7
index 687f7dd7e13a073314210d1ea243891bdb8b26fa..2ffea3b50a9b4211fa8110f989822d9ac3b58ee9 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7
@@ -1 +1 @@
-c=France/d=4
\ No newline at end of file
+c=France/d=4
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6
index 1329d173d6a21d901d6462cb091db36a5bd352cd..ca4194fbcf3ff4eb17fc0b6791bb6af030049fa4 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6
@@ -1,10 +1,10 @@
-a                   	string              	None                
-b                   	int                 	None                
-c                   	int                 	None                
-d                   	int                 	None                
+a                   	string              	                    
+b                   	int                 	                    
+c                   	int                 	                    
+d                   	int                 	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-c                   	int                 	None                
-d                   	int                 	None                
\ No newline at end of file
+c                   	int                 	                    
+d                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7
index 08051a26d24cc9726534d4e48a39d85a09d003c5..7ace4dc6623064e0594adf597d17d420bf3b0271 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7
@@ -2,4 +2,4 @@ c=1/d=2
 c=2/d=1
 c=2/d=2
 c=3/d=1
-c=3/d=2
\ No newline at end of file
+c=30/d=2
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7
index 7f6e4ae8abf835de4170296c29a36cd2c913029b..1ca1833c09245a092031e6a7d888036e5fe5a0e8 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7
@@ -1,3 +1,3 @@
 c=1/d=2
 c=3/d=1
-c=3/d=2
\ No newline at end of file
+c=30/d=2
diff --git a/sql/hive/src/test/resources/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-13-8117981303487dc4c4873356931ef26a
similarity index 100%
rename from sql/hive/src/test/resources/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9
rename to sql/hive/src/test/resources/golden/drop_partitions_filter2-13-8117981303487dc4c4873356931ef26a
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-14-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-14-83e3e422cdf9403523fa60d75376d7d7
new file mode 100644
index 0000000000000000000000000000000000000000..fee1b1ad01412a2aca79e7073ffe80e62383b35a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-14-83e3e422cdf9403523fa60d75376d7d7
@@ -0,0 +1 @@
+c=30/d=2
diff --git a/sql/hive/src/test/resources/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba b/sql/hive/src/test/resources/golden/drop_partitions_filter2-15-67d75c72ea2d3982c55f3a850d93f83c
similarity index 100%
rename from sql/hive/src/test/resources/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba
rename to sql/hive/src/test/resources/golden/drop_partitions_filter2-15-67d75c72ea2d3982c55f3a850d93f83c
diff --git a/sql/hive/src/test/resources/golden/database_drop-8-97101266791d2b2c662bcde549422318 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-7-f34625fd49a5e655cba3abb5cb8c5417
similarity index 100%
rename from sql/hive/src/test/resources/golden/database_drop-8-97101266791d2b2c662bcde549422318
rename to sql/hive/src/test/resources/golden/drop_partitions_filter2-7-f34625fd49a5e655cba3abb5cb8c5417
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7
index 226ef460b53a68290abd5e51cf9f5dceadc14147..b77f18cd02020b79d11c74f68c45d0b9fc4d383d 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7
@@ -3,4 +3,4 @@ c=1/d=2
 c=2/d=1
 c=2/d=2
 c=3/d=1
-c=3/d=2
\ No newline at end of file
+c=30/d=2
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6
index 01562f65d807fa04e9360b0fdf53895da1f68b26..77bc36b96870b9a3475d0e714d84bb4e16906079 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6
@@ -1,10 +1,10 @@
-a                   	string              	None                
-b                   	int                 	None                
-c                   	string              	None                
-d                   	int                 	None                
+a                   	string              	                    
+b                   	int                 	                    
+c                   	string              	                    
+d                   	int                 	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-c                   	string              	None                
-d                   	int                 	None                
\ No newline at end of file
+c                   	string              	                    
+d                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7
index 08051a26d24cc9726534d4e48a39d85a09d003c5..50e8df00f15973abbc12e4025cac626f18ba2a9b 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7
@@ -2,4 +2,4 @@ c=1/d=2
 c=2/d=1
 c=2/d=2
 c=3/d=1
-c=3/d=2
\ No newline at end of file
+c=3/d=2
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7
index 7f6e4ae8abf835de4170296c29a36cd2c913029b..20bc2b0c74d325d2457321e16db95bddfa6f624d 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7
@@ -1,3 +1,3 @@
 c=1/d=2
 c=3/d=1
-c=3/d=2
\ No newline at end of file
+c=3/d=2
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7
index 226ef460b53a68290abd5e51cf9f5dceadc14147..6200b3ad2dd9482f64415cdffd9d0de4082860ae 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7
+++ b/sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7
@@ -3,4 +3,4 @@ c=1/d=2
 c=2/d=1
 c=2/d=2
 c=3/d=1
-c=3/d=2
\ No newline at end of file
+c=3/d=2
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 b/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1
index 8caab1c99b27dc70f1c41749dca0c852d6cd33ab..73f873dbcf00b36ee6b326b7fb4bc814528678ac 100644
--- a/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1
+++ b/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1
@@ -1,10 +1,10 @@
-c1                  	string              	None                
-c2                  	string              	None                
-p                   	string              	None                
+c1                  	string              	                    
+c2                  	string              	                    
+p                   	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-p                   	string              	None                
+p                   	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[p1], dbName:default, tableName:tbl_protectmode_no_drop, createTime:1389728724, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:c1, type:string, comment:null), FieldSchema(name:c2, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/tbl_protectmode_no_drop/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1389728724, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1389728724})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[p1], dbName:default, tableName:tbl_protectmode_no_drop, createTime:1413879999, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:c1, type:string, comment:null), FieldSchema(name:c2, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl_protectmode_no_drop/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413879999, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1413879999, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1})	
diff --git a/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48
+++ b/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 b/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2
+++ b/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 b/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4
index fbe12dca4efc86e272fedf84d7e706b277b19c7f..a4b5a454432353662238d50d4624d9b5b7497bc7 100644
--- a/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4
+++ b/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4
@@ -1,3 +1,3 @@
 p=p1
 p=p2
-p=p3
\ No newline at end of file
+p=p3
diff --git a/sql/hive/src/test/resources/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-0-97b52abf021c81b8364041c1a0bbccf3
similarity index 100%
rename from sql/hive/src/test/resources/golden/database_drop-9-8db536f925bf0f5058f97897e145a661
rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-0-97b52abf021c81b8364041c1a0bbccf3
diff --git a/sql/hive/src/test/resources/golden/date_2-2-cab14d992c53c106ab257fae52001e04 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-1-f11a45c42752d06821ccd26d948d51ff
similarity index 100%
rename from sql/hive/src/test/resources/golden/date_2-2-cab14d992c53c106ab257fae52001e04
rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-1-f11a45c42752d06821ccd26d948d51ff
diff --git a/sql/hive/src/test/resources/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-2-c0b85445b616f93c5e6d090fa35072e7
similarity index 100%
rename from sql/hive/src/test/resources/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8
rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-2-c0b85445b616f93c5e6d090fa35072e7
diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-3-10a71bca930d911cc4c2022575b17299
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-3-10a71bca930d911cc4c2022575b17299
diff --git a/sql/hive/src/test/resources/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-4-b2ca31dd6cc5c32e33df700786f5b208
similarity index 100%
rename from sql/hive/src/test/resources/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d
rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-4-b2ca31dd6cc5c32e33df700786f5b208
diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-5-10a71bca930d911cc4c2022575b17299
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-5-10a71bca930d911cc4c2022575b17299
diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-6-d1c175a9d042ecd389f2f93fc867591d
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-6-d1c175a9d042ecd389f2f93fc867591d
diff --git a/sql/hive/src/test/resources/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-3cd14bc5b126ff8b337c4abc09134260
similarity index 100%
rename from sql/hive/src/test/resources/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3
rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-3cd14bc5b126ff8b337c4abc09134260
diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe
+++ b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-ce929ee6a92b81d8080ca322c1c38a4b
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4
rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-ce929ee6a92b81d8080ca322c1c38a4b
diff --git a/sql/hive/src/test/resources/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-f3a5a998099b756a21cf9122a15b09d5
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd
rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-f3a5a998099b756a21cf9122a15b09d5
diff --git a/sql/hive/src/test/resources/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-c8aa1ebce4b0b2b7f46bb3c2502f8b49
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4
rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-c8aa1ebce4b0b2b7f46bb3c2502f8b49
diff --git a/sql/hive/src/test/resources/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-fed732d2e7d94a4fc02e7694f9f9a39c
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653
rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-fed732d2e7d94a4fc02e7694f9f9a39c
diff --git a/sql/hive/src/test/resources/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-adab6a0187003ab7ee6f217c9e409d91
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5
rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-adab6a0187003ab7ee6f217c9e409d91
diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1
+++ b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-adab6a0187003ab7ee6f217c9e409d91
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee
rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-adab6a0187003ab7ee6f217c9e409d91
diff --git a/sql/hive/src/test/resources/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-4b099f87e221b8fd5c0d0d4a97c0d146
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b
rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-4b099f87e221b8fd5c0d0d4a97c0d146
diff --git a/sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf b/sql/hive/src/test/resources/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf
deleted file mode 100644
index 8273b7ed19da6456d197043c12236fb71a5b106d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf
+++ /dev/null
@@ -1,10 +0,0 @@
-0	val_0
-0	val_0
-0	val_0
-10	val_10
-100	val_100
-100	val_100
-103	val_103
-103	val_103
-104	val_104
-104	val_104
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 b/sql/hive/src/test/resources/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314
deleted file mode 100644
index 045906a29a1da3335b9b3588a2be39161c699de2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314
+++ /dev/null
@@ -1,10 +0,0 @@
-98	val_98
-98	val_98
-97	val_97
-97	val_97
-96	val_96
-95	val_95
-95	val_95
-92	val_92
-90	val_90
-90	val_90
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 b/sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960
deleted file mode 100644
index a949a93dfcca659638ff00284e1d1703accef77a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960
+++ /dev/null
@@ -1 +0,0 @@
-128
diff --git a/sql/hive/src/test/resources/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 b/sql/hive/src/test/resources/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17
deleted file mode 100644
index e2799aa7e14b5885aad48117308f489740683449..0000000000000000000000000000000000000000
Binary files a/sql/hive/src/test/resources/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 and /dev/null differ
diff --git a/sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 b/sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960
deleted file mode 100644
index a949a93dfcca659638ff00284e1d1703accef77a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960
+++ /dev/null
@@ -1 +0,0 @@
-128
diff --git a/sql/hive/src/test/resources/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 b/sql/hive/src/test/resources/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17
deleted file mode 100644
index e2799aa7e14b5885aad48117308f489740683449..0000000000000000000000000000000000000000
Binary files a/sql/hive/src/test/resources/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 and /dev/null differ
diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d b/sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 b/sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 b/sql/hive/src/test/resources/golden/explain_rearrange-0-6f7c8515c354fb050829ebd66413425
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035
rename to sql/hive/src/test/resources/golden/explain_rearrange-0-6f7c8515c354fb050829ebd66413425
diff --git a/sql/hive/src/test/resources/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 b/sql/hive/src/test/resources/golden/explain_rearrange-1-378d42317b39c6519f15bd2f99c5ddc4
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506
rename to sql/hive/src/test/resources/golden/explain_rearrange-1-378d42317b39c6519f15bd2f99c5ddc4
diff --git a/sql/hive/src/test/resources/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e b/sql/hive/src/test/resources/golden/explain_rearrange-10-3f2680208772a0e51aefc4ef5604dddf
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e
rename to sql/hive/src/test/resources/golden/explain_rearrange-10-3f2680208772a0e51aefc4ef5604dddf
diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/explain_rearrange-11-f2ca12a948fd9b5b842168e7c7d7b768
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/explain_rearrange-11-f2ca12a948fd9b5b842168e7c7d7b768
diff --git a/sql/hive/src/test/resources/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee b/sql/hive/src/test/resources/golden/explain_rearrange-12-3d63f0bb8fbacbcff9e5989ddf1bcc8e
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee
rename to sql/hive/src/test/resources/golden/explain_rearrange-12-3d63f0bb8fbacbcff9e5989ddf1bcc8e
diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/explain_rearrange-13-5baad22ed7efa18d73eb8349e57cf331
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/explain_rearrange-13-5baad22ed7efa18d73eb8349e57cf331
diff --git a/sql/hive/src/test/resources/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef b/sql/hive/src/test/resources/golden/explain_rearrange-14-490d6253b73064ce403e4d04a8bc18f3
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef
rename to sql/hive/src/test/resources/golden/explain_rearrange-14-490d6253b73064ce403e4d04a8bc18f3
diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/explain_rearrange-2-24ca942f094b14b92086305cc125e833
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/explain_rearrange-2-24ca942f094b14b92086305cc125e833
diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/explain_rearrange-3-3b0f76816be2c1b18a2058027a19bc9f
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/explain_rearrange-3-3b0f76816be2c1b18a2058027a19bc9f
diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 b/sql/hive/src/test/resources/golden/explain_rearrange-4-86473a0498e4361e4db0b4a22f2e8571
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394
rename to sql/hive/src/test/resources/golden/explain_rearrange-4-86473a0498e4361e4db0b4a22f2e8571
diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/explain_rearrange-5-d0ec6d66ff349db09fd455eec149efdb
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/explain_rearrange-5-d0ec6d66ff349db09fd455eec149efdb
diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/explain_rearrange-6-cda81d86d127fca0e2fbc2161e91400d
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/explain_rearrange-6-cda81d86d127fca0e2fbc2161e91400d
diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/explain_rearrange-7-5b13cfa4b730e38ef2794c1532968e04
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/explain_rearrange-7-5b13cfa4b730e38ef2794c1532968e04
diff --git a/sql/hive/src/test/resources/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 b/sql/hive/src/test/resources/golden/explain_rearrange-8-1fd9c02fc67c3a403cb73eb10ed9fc12
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01
rename to sql/hive/src/test/resources/golden/explain_rearrange-8-1fd9c02fc67c3a403cb73eb10ed9fc12
diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/explain_rearrange-9-73b9ac83dbc9874dc9379ad4364d40ac
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/explain_rearrange-9-73b9ac83dbc9874dc9379ad4364d40ac
diff --git a/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 b/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607
+++ b/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 b/sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7
index fb12634ea81c1a87a97eee102fa335314906b4e8..8059361d2485f3c18afe78234fee92d61afe49bd 100644
--- a/sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7
+++ b/sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7
@@ -1 +1 @@
-500	130091	260.182	0	498	142.92680950752384	20428.072876000006
\ No newline at end of file
+500	130091	260.182	0	498	142.92680950752384	20428.072876000006
diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 b/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279
+++ b/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 b/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818
index d23e05acf7ba58c8471721a7bb93d585e57e75c8..e34118512c1d769075d0b492b217316096597c08 100644
--- a/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818
+++ b/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818
@@ -497,4 +497,4 @@
 403
 400
 200
-97
\ No newline at end of file
+97
diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 b/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131
index 6280b32facd66657a8b0f766845da281eb93113e..1cf9f21c9ed8800e5f6228dd8a72b94174b16ec4 100644
--- a/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131
+++ b/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131
@@ -1,4 +1,4 @@
-key                 	int                 	None                
-value               	string              	None                
+key                 	int                 	                    
+value               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1398823397, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/dest1, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1398823397}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1413880056, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/dest1, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413880056}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 b/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131
index e793ec2f946e57d9152a3d0cfd5cb78bf57a3d24..62d6734063fdd8bd5a7800759b865161cc9a3c6a 100644
--- a/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131
+++ b/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131
@@ -1,4 +1,4 @@
-key                 	int                 	None                
-value               	string              	None                
+key                 	int                 	                    
+value               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1398823407, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/dest1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1398823407}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1413880064, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/dest1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413880064}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e b/sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e
index cc545367b951b94e8faabc280ba476f761c1f974..31956a614026ddc13336aea35bdb6e94d8fadaba 100644
--- a/sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e
+++ b/sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e
@@ -9,4 +9,4 @@
 66	val_66
 213	val_213
 146	val_146
-406	val_406
\ No newline at end of file
+406	val_406
diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd b/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd
index 140c6590a27fccf3f8e65eacb6b6bd3cb1edddcf..51eb2d30c97eb78529a8f225d7b0636b1e60d6e5 100644
--- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd
+++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd
@@ -1 +1 @@
-4	5	0	2010-04-17
\ No newline at end of file
+4	5	0	2010-04-17
diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370
index 51f5701ceae460100b988d3fa7ab9e5f3a7686bd..7dc9c13cfee044980f07d1d0edefc881b075ed7d 100644
--- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370
+++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370
@@ -1 +1 @@
-4	1	1	8	4	5	1	0	9	U	2	2	0	2	1	1	J	C	A	U	2	s	2	NULL	NULL	NULL	NULL	NULL	NULL	1	j	S	6	NULL	1	2	J	g	1	e	2	1	2	U	P	p	3	0	0	0	1	1	1	0	0	0	6	2	j	NULL	NULL	NULL	NULL	NULL	NULL	5	NULL	NULL	j	2	2	1	2	2	1	1	1	1	1	1	1	1	32	NULL	2010-04-17
\ No newline at end of file
+4	1	1	8	4	5	1	0	9	U	2	2	0	2	1	1	J	C	A	U	2	s	2	NULL	NULL	NULL	NULL	NULL	NULL	1	j	S	6	NULL	1	2	J	g	1	e	2	1	2	U	P	p	3	0	0	0	1	1	1	0	0	0	6	2	j	NULL	NULL	NULL	NULL	NULL	NULL	5	NULL	NULL	j	2	2	1	2	2	1	1	1	1	1	1	1	1	32	NULL	2010-04-17
diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179
index 69dcc336a73f211436b515b0f7db66a377152a41..19a6df9f78cd7b58eeb4e8a20ac78a3df4ac1403 100644
--- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179
+++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179
@@ -1 +1 @@
-5	5	4
\ No newline at end of file
+5	5	4
diff --git a/sql/hive/src/test/resources/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-4-fe8b55e4e4098d7a2662338783a50306
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792
rename to sql/hive/src/test/resources/golden/filter_join_breaktask2-4-fe8b55e4e4098d7a2662338783a50306
diff --git a/sql/hive/src/test/resources/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-5-cf724251613216ec54f8ac2e6b9b92fd
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104
rename to sql/hive/src/test/resources/golden/filter_join_breaktask2-5-cf724251613216ec54f8ac2e6b9b92fd
diff --git a/sql/hive/src/test/resources/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d b/sql/hive/src/test/resources/golden/filter_join_breaktask2-6-8c782ae8f8245bdbe90d068a6b577d1e
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d
rename to sql/hive/src/test/resources/golden/filter_join_breaktask2-6-8c782ae8f8245bdbe90d068a6b577d1e
diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e b/sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e
index c0f577c3cd2fd945771755435c5d314b2bbcc460..d6961f6e46e937453ca797a6cb3aaad4de926ac0 100644
--- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e
+++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e
@@ -1 +1 @@
-5	name	NULL	2	kavin	NULL	9	c	8	0	0	7	1	2	0	3	2	NULL	1	NULL	3	2	0	0	5	10	2010-04-17
\ No newline at end of file
+5	name	NULL	2	kavin	NULL	9	c	8	0	0	7	1	2	0	3	2	NULL	1	NULL	3	2	0	0	5	10	2010-04-17
diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1
index e426b4879bcb6a3872dc33423d0252b93d68a9c5..be23778bd2cc488d3809eca4dd2334044d9b9a30 100644
--- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1
+++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1
@@ -1 +1 @@
-5	1	1	1	0	0	4	2010-04-17
\ No newline at end of file
+5	1	1	1	0	0	4	2010-04-17
diff --git a/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d b/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d
+++ b/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 b/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168
+++ b/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 b/sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..4953d093489ee79f5dd9f0cbcd91375cc0db53e9 100644
--- a/sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7
+++ b/sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7
@@ -306,4 +306,4 @@
 495	495.0
 496	496.0
 497	497.0
-498	1494.0
\ No newline at end of file
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd b/sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd
index 8939d9f8af186d7d3efe74b8ef11d547c12acb4a..07df09912b06e44127748f216e8c3ca397c426cf 100644
--- a/sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd
+++ b/sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd
@@ -306,4 +306,4 @@ val_92	1	1	111
 val_95	2	1	111
 val_96	1	1	111
 val_97	2	1	111
-val_98	2	1	111
\ No newline at end of file
+val_98	2	1	111
diff --git a/sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d b/sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d
index 82a0329b4d459b4c583e56672bfe99eed74d6bd9..efbfb713807a025dcda948f48168ac9f812a3100 100644
--- a/sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d
+++ b/sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d
@@ -306,4 +306,4 @@
 95	2	1	111
 96	1	1	111
 97	2	1	111
-98	2	1	111
\ No newline at end of file
+98	2	1	111
diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/groupby12-0-67e864faaff4c6b2a8e1c9fbd188bb66
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/groupby12-0-67e864faaff4c6b2a8e1c9fbd188bb66
diff --git a/sql/hive/src/test/resources/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e b/sql/hive/src/test/resources/golden/groupby12-1-13ab74a58da514fe01dbeda0c3e79883
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e
rename to sql/hive/src/test/resources/golden/groupby12-1-13ab74a58da514fe01dbeda0c3e79883
diff --git a/sql/hive/src/test/resources/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c b/sql/hive/src/test/resources/golden/groupby12-2-fd150794945892f3c926a1881cd819f4
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c
rename to sql/hive/src/test/resources/golden/groupby12-2-fd150794945892f3c926a1881cd819f4
diff --git a/sql/hive/src/test/resources/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 b/sql/hive/src/test/resources/golden/groupby12-3-8018bb917a0706925c14421ec2761663
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4
rename to sql/hive/src/test/resources/golden/groupby12-3-8018bb917a0706925c14421ec2761663
diff --git a/sql/hive/src/test/resources/golden/groupby12-4-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby12-4-adc1ec67836b26b60d8547c4996bfd8f
new file mode 100644
index 0000000000000000000000000000000000000000..98f8836673e8a767eae1846d787722516f077d72
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby12-4-adc1ec67836b26b60d8547c4996bfd8f
@@ -0,0 +1,309 @@
+3	1
+1	1
+1	1
+3	1
+1	1
+1	1
+1	1
+1	1
+2	1
+2	1
+1	1
+2	1
+1	1
+1	1
+2	1
+2	1
+1	1
+1	1
+1	1
+1	1
+1	1
+3	1
+2	1
+1	1
+2	1
+1	1
+1	1
+1	1
+2	1
+1	1
+1	1
+1	1
+2	1
+1	1
+1	1
+1	1
+2	1
+1	1
+3	1
+2	1
+1	1
+2	1
+1	1
+1	1
+1	1
+1	1
+2	1
+2	1
+1	1
+1	1
+1	1
+3	1
+1	1
+2	1
+1	1
+2	1
+2	1
+2	1
+2	1
+2	1
+1	1
+1	1
+2	1
+1	1
+1	1
+2	1
+3	1
+2	1
+2	1
+1	1
+3	1
+2	1
+1	1
+1	1
+2	1
+1	1
+2	1
+4	1
+1	1
+1	1
+2	1
+2	1
+1	1
+2	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+2	1
+2	1
+1	1
+3	1
+1	1
+4	1
+1	1
+2	1
+2	1
+2	1
+2	1
+1	1
+1	1
+2	1
+1	1
+1	1
+1	1
+1	1
+3	1
+1	1
+1	1
+2	1
+1	1
+3	1
+1	1
+2	1
+1	1
+2	1
+3	1
+2	1
+1	1
+1	1
+2	1
+2	1
+2	1
+3	1
+2	1
+2	1
+1	1
+2	1
+2	1
+1	1
+2	1
+2	1
+1	1
+2	1
+2	1
+1	1
+1	1
+2	1
+5	1
+2	1
+1	1
+2	1
+2	1
+2	1
+1	1
+2	1
+1	1
+1	1
+1	1
+1	1
+1	1
+2	1
+2	1
+1	1
+1	1
+1	1
+1	1
+1	1
+2	1
+1	1
+2	1
+3	1
+1	1
+1	1
+4	1
+2	1
+2	1
+2	1
+2	1
+1	1
+1	1
+1	1
+1	1
+1	1
+2	1
+1	1
+1	1
+1	1
+1	1
+3	1
+1	1
+1	1
+1	1
+2	1
+1	1
+2	1
+1	1
+3	1
+1	1
+3	1
+2	1
+3	1
+2	1
+2	1
+1	1
+2	1
+3	1
+2	1
+1	1
+2	1
+1	1
+1	1
+1	1
+1	1
+1	1
+2	1
+2	1
+1	1
+5	1
+1	1
+2	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+2	1
+1	1
+3	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+2	1
+3	1
+1	1
+1	1
+1	1
+1	1
+1	1
+2	1
+3	1
+2	1
+2	1
+1	1
+5	1
+1	1
+3	1
+2	1
+4	1
+1	1
+3	1
+1	1
+2	1
+2	1
+3	1
+1	1
+1	1
+1	1
+2	1
+1	1
+2	1
+3	1
+3	1
+1	1
+1	1
+1	1
+1	1
+3	1
+2	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+3	1
+1	1
+1	1
+2	1
+2	1
+1	1
+2	1
+2	1
+3	1
+1	1
+4	1
+5	1
+1	1
+1	1
+1	1
+1	1
+2	1
+1	1
+3	1
+1	1
+1	1
+1	1
+1	1
+1	1
+1	1
+4	1
+1	1
+1	1
+2	1
+1	1
+1	1
+1	1
+1	1
+1	1
+3	1
diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e b/sql/hive/src/test/resources/golden/groupby1_limit-3-607512a8dd16cd9ddde561eeabfa51db
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e
rename to sql/hive/src/test/resources/golden/groupby1_limit-3-607512a8dd16cd9ddde561eeabfa51db
diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 b/sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2
index 9cf9606d751ac1af6936ba0a62dc3102b77060a2..f92f60d11d4ef5131ff0e80ba3e176d3a52d3690 100644
--- a/sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2
+++ b/sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2
@@ -2,4 +2,4 @@
 2	2.0
 4	4.0
 5	15.0
-8	8.0
\ No newline at end of file
+8	8.0
diff --git a/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,309 +0,0 @@
-0	0.0
-2	2.0
-4	4.0
-5	15.0
-8	8.0
-9	9.0
-10	10.0
-11	11.0
-12	24.0
-15	30.0
-17	17.0
-18	36.0
-19	19.0
-20	20.0
-24	48.0
-26	52.0
-27	27.0
-28	28.0
-30	30.0
-33	33.0
-34	34.0
-35	105.0
-37	74.0
-41	41.0
-42	84.0
-43	43.0
-44	44.0
-47	47.0
-51	102.0
-53	53.0
-54	54.0
-57	57.0
-58	116.0
-64	64.0
-65	65.0
-66	66.0
-67	134.0
-69	69.0
-70	210.0
-72	144.0
-74	74.0
-76	152.0
-77	77.0
-78	78.0
-80	80.0
-82	82.0
-83	166.0
-84	168.0
-85	85.0
-86	86.0
-87	87.0
-90	270.0
-92	92.0
-95	190.0
-96	96.0
-97	194.0
-98	196.0
-100	200.0
-103	206.0
-104	208.0
-105	105.0
-111	111.0
-113	226.0
-114	114.0
-116	116.0
-118	236.0
-119	357.0
-120	240.0
-125	250.0
-126	126.0
-128	384.0
-129	258.0
-131	131.0
-133	133.0
-134	268.0
-136	136.0
-137	274.0
-138	552.0
-143	143.0
-145	145.0
-146	292.0
-149	298.0
-150	150.0
-152	304.0
-153	153.0
-155	155.0
-156	156.0
-157	157.0
-158	158.0
-160	160.0
-162	162.0
-163	163.0
-164	328.0
-165	330.0
-166	166.0
-167	501.0
-168	168.0
-169	676.0
-170	170.0
-172	344.0
-174	348.0
-175	350.0
-176	352.0
-177	177.0
-178	178.0
-179	358.0
-180	180.0
-181	181.0
-183	183.0
-186	186.0
-187	561.0
-189	189.0
-190	190.0
-191	382.0
-192	192.0
-193	579.0
-194	194.0
-195	390.0
-196	196.0
-197	394.0
-199	597.0
-200	400.0
-201	201.0
-202	202.0
-203	406.0
-205	410.0
-207	414.0
-208	624.0
-209	418.0
-213	426.0
-214	214.0
-216	432.0
-217	434.0
-218	218.0
-219	438.0
-221	442.0
-222	222.0
-223	446.0
-224	448.0
-226	226.0
-228	228.0
-229	458.0
-230	1150.0
-233	466.0
-235	235.0
-237	474.0
-238	476.0
-239	478.0
-241	241.0
-242	484.0
-244	244.0
-247	247.0
-248	248.0
-249	249.0
-252	252.0
-255	510.0
-256	512.0
-257	257.0
-258	258.0
-260	260.0
-262	262.0
-263	263.0
-265	530.0
-266	266.0
-272	544.0
-273	819.0
-274	274.0
-275	275.0
-277	1108.0
-278	556.0
-280	560.0
-281	562.0
-282	564.0
-283	283.0
-284	284.0
-285	285.0
-286	286.0
-287	287.0
-288	576.0
-289	289.0
-291	291.0
-292	292.0
-296	296.0
-298	894.0
-302	302.0
-305	305.0
-306	306.0
-307	614.0
-308	308.0
-309	618.0
-310	310.0
-311	933.0
-315	315.0
-316	948.0
-317	634.0
-318	954.0
-321	642.0
-322	644.0
-323	323.0
-325	650.0
-327	981.0
-331	662.0
-332	332.0
-333	666.0
-335	335.0
-336	336.0
-338	338.0
-339	339.0
-341	341.0
-342	684.0
-344	688.0
-345	345.0
-348	1740.0
-351	351.0
-353	706.0
-356	356.0
-360	360.0
-362	362.0
-364	364.0
-365	365.0
-366	366.0
-367	734.0
-368	368.0
-369	1107.0
-373	373.0
-374	374.0
-375	375.0
-377	377.0
-378	378.0
-379	379.0
-382	764.0
-384	1152.0
-386	386.0
-389	389.0
-392	392.0
-393	393.0
-394	394.0
-395	790.0
-396	1188.0
-397	794.0
-399	798.0
-400	400.0
-401	2005.0
-402	402.0
-403	1209.0
-404	808.0
-406	1624.0
-407	407.0
-409	1227.0
-411	411.0
-413	826.0
-414	828.0
-417	1251.0
-418	418.0
-419	419.0
-421	421.0
-424	848.0
-427	427.0
-429	858.0
-430	1290.0
-431	1293.0
-432	432.0
-435	435.0
-436	436.0
-437	437.0
-438	1314.0
-439	878.0
-443	443.0
-444	444.0
-446	446.0
-448	448.0
-449	449.0
-452	452.0
-453	453.0
-454	1362.0
-455	455.0
-457	457.0
-458	916.0
-459	918.0
-460	460.0
-462	924.0
-463	926.0
-466	1398.0
-467	467.0
-468	1872.0
-469	2345.0
-470	470.0
-472	472.0
-475	475.0
-477	477.0
-478	956.0
-479	479.0
-480	1440.0
-481	481.0
-482	482.0
-483	483.0
-484	484.0
-485	485.0
-487	487.0
-489	1956.0
-490	490.0
-491	491.0
-492	984.0
-493	493.0
-494	494.0
-495	495.0
-496	496.0
-497	497.0
-498	1494.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby1_map-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby1_map-6-c83b156356c269b6aef263640a4f6b7b
new file mode 100644
index 0000000000000000000000000000000000000000..4953d093489ee79f5dd9f0cbcd91375cc0db53e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby1_map-6-c83b156356c269b6aef263640a4f6b7b
@@ -0,0 +1,309 @@
+0	0.0
+2	2.0
+4	4.0
+5	15.0
+8	8.0
+9	9.0
+10	10.0
+11	11.0
+12	24.0
+15	30.0
+17	17.0
+18	36.0
+19	19.0
+20	20.0
+24	48.0
+26	52.0
+27	27.0
+28	28.0
+30	30.0
+33	33.0
+34	34.0
+35	105.0
+37	74.0
+41	41.0
+42	84.0
+43	43.0
+44	44.0
+47	47.0
+51	102.0
+53	53.0
+54	54.0
+57	57.0
+58	116.0
+64	64.0
+65	65.0
+66	66.0
+67	134.0
+69	69.0
+70	210.0
+72	144.0
+74	74.0
+76	152.0
+77	77.0
+78	78.0
+80	80.0
+82	82.0
+83	166.0
+84	168.0
+85	85.0
+86	86.0
+87	87.0
+90	270.0
+92	92.0
+95	190.0
+96	96.0
+97	194.0
+98	196.0
+100	200.0
+103	206.0
+104	208.0
+105	105.0
+111	111.0
+113	226.0
+114	114.0
+116	116.0
+118	236.0
+119	357.0
+120	240.0
+125	250.0
+126	126.0
+128	384.0
+129	258.0
+131	131.0
+133	133.0
+134	268.0
+136	136.0
+137	274.0
+138	552.0
+143	143.0
+145	145.0
+146	292.0
+149	298.0
+150	150.0
+152	304.0
+153	153.0
+155	155.0
+156	156.0
+157	157.0
+158	158.0
+160	160.0
+162	162.0
+163	163.0
+164	328.0
+165	330.0
+166	166.0
+167	501.0
+168	168.0
+169	676.0
+170	170.0
+172	344.0
+174	348.0
+175	350.0
+176	352.0
+177	177.0
+178	178.0
+179	358.0
+180	180.0
+181	181.0
+183	183.0
+186	186.0
+187	561.0
+189	189.0
+190	190.0
+191	382.0
+192	192.0
+193	579.0
+194	194.0
+195	390.0
+196	196.0
+197	394.0
+199	597.0
+200	400.0
+201	201.0
+202	202.0
+203	406.0
+205	410.0
+207	414.0
+208	624.0
+209	418.0
+213	426.0
+214	214.0
+216	432.0
+217	434.0
+218	218.0
+219	438.0
+221	442.0
+222	222.0
+223	446.0
+224	448.0
+226	226.0
+228	228.0
+229	458.0
+230	1150.0
+233	466.0
+235	235.0
+237	474.0
+238	476.0
+239	478.0
+241	241.0
+242	484.0
+244	244.0
+247	247.0
+248	248.0
+249	249.0
+252	252.0
+255	510.0
+256	512.0
+257	257.0
+258	258.0
+260	260.0
+262	262.0
+263	263.0
+265	530.0
+266	266.0
+272	544.0
+273	819.0
+274	274.0
+275	275.0
+277	1108.0
+278	556.0
+280	560.0
+281	562.0
+282	564.0
+283	283.0
+284	284.0
+285	285.0
+286	286.0
+287	287.0
+288	576.0
+289	289.0
+291	291.0
+292	292.0
+296	296.0
+298	894.0
+302	302.0
+305	305.0
+306	306.0
+307	614.0
+308	308.0
+309	618.0
+310	310.0
+311	933.0
+315	315.0
+316	948.0
+317	634.0
+318	954.0
+321	642.0
+322	644.0
+323	323.0
+325	650.0
+327	981.0
+331	662.0
+332	332.0
+333	666.0
+335	335.0
+336	336.0
+338	338.0
+339	339.0
+341	341.0
+342	684.0
+344	688.0
+345	345.0
+348	1740.0
+351	351.0
+353	706.0
+356	356.0
+360	360.0
+362	362.0
+364	364.0
+365	365.0
+366	366.0
+367	734.0
+368	368.0
+369	1107.0
+373	373.0
+374	374.0
+375	375.0
+377	377.0
+378	378.0
+379	379.0
+382	764.0
+384	1152.0
+386	386.0
+389	389.0
+392	392.0
+393	393.0
+394	394.0
+395	790.0
+396	1188.0
+397	794.0
+399	798.0
+400	400.0
+401	2005.0
+402	402.0
+403	1209.0
+404	808.0
+406	1624.0
+407	407.0
+409	1227.0
+411	411.0
+413	826.0
+414	828.0
+417	1251.0
+418	418.0
+419	419.0
+421	421.0
+424	848.0
+427	427.0
+429	858.0
+430	1290.0
+431	1293.0
+432	432.0
+435	435.0
+436	436.0
+437	437.0
+438	1314.0
+439	878.0
+443	443.0
+444	444.0
+446	446.0
+448	448.0
+449	449.0
+452	452.0
+453	453.0
+454	1362.0
+455	455.0
+457	457.0
+458	916.0
+459	918.0
+460	460.0
+462	924.0
+463	926.0
+466	1398.0
+467	467.0
+468	1872.0
+469	2345.0
+470	470.0
+472	472.0
+475	475.0
+477	477.0
+478	956.0
+479	479.0
+480	1440.0
+481	481.0
+482	482.0
+483	483.0
+484	484.0
+485	485.0
+487	487.0
+489	1956.0
+490	490.0
+491	491.0
+492	984.0
+493	493.0
+494	494.0
+495	495.0
+496	496.0
+497	497.0
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 b/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4
+++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..4953d093489ee79f5dd9f0cbcd91375cc0db53e9 100644
--- a/sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f
@@ -306,4 +306,4 @@
 495	495.0
 496	496.0
 497	497.0
-498	1494.0
\ No newline at end of file
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,309 +0,0 @@
-0	0.0
-2	2.0
-4	4.0
-5	15.0
-8	8.0
-9	9.0
-10	10.0
-11	11.0
-12	24.0
-15	30.0
-17	17.0
-18	36.0
-19	19.0
-20	20.0
-24	48.0
-26	52.0
-27	27.0
-28	28.0
-30	30.0
-33	33.0
-34	34.0
-35	105.0
-37	74.0
-41	41.0
-42	84.0
-43	43.0
-44	44.0
-47	47.0
-51	102.0
-53	53.0
-54	54.0
-57	57.0
-58	116.0
-64	64.0
-65	65.0
-66	66.0
-67	134.0
-69	69.0
-70	210.0
-72	144.0
-74	74.0
-76	152.0
-77	77.0
-78	78.0
-80	80.0
-82	82.0
-83	166.0
-84	168.0
-85	85.0
-86	86.0
-87	87.0
-90	270.0
-92	92.0
-95	190.0
-96	96.0
-97	194.0
-98	196.0
-100	200.0
-103	206.0
-104	208.0
-105	105.0
-111	111.0
-113	226.0
-114	114.0
-116	116.0
-118	236.0
-119	357.0
-120	240.0
-125	250.0
-126	126.0
-128	384.0
-129	258.0
-131	131.0
-133	133.0
-134	268.0
-136	136.0
-137	274.0
-138	552.0
-143	143.0
-145	145.0
-146	292.0
-149	298.0
-150	150.0
-152	304.0
-153	153.0
-155	155.0
-156	156.0
-157	157.0
-158	158.0
-160	160.0
-162	162.0
-163	163.0
-164	328.0
-165	330.0
-166	166.0
-167	501.0
-168	168.0
-169	676.0
-170	170.0
-172	344.0
-174	348.0
-175	350.0
-176	352.0
-177	177.0
-178	178.0
-179	358.0
-180	180.0
-181	181.0
-183	183.0
-186	186.0
-187	561.0
-189	189.0
-190	190.0
-191	382.0
-192	192.0
-193	579.0
-194	194.0
-195	390.0
-196	196.0
-197	394.0
-199	597.0
-200	400.0
-201	201.0
-202	202.0
-203	406.0
-205	410.0
-207	414.0
-208	624.0
-209	418.0
-213	426.0
-214	214.0
-216	432.0
-217	434.0
-218	218.0
-219	438.0
-221	442.0
-222	222.0
-223	446.0
-224	448.0
-226	226.0
-228	228.0
-229	458.0
-230	1150.0
-233	466.0
-235	235.0
-237	474.0
-238	476.0
-239	478.0
-241	241.0
-242	484.0
-244	244.0
-247	247.0
-248	248.0
-249	249.0
-252	252.0
-255	510.0
-256	512.0
-257	257.0
-258	258.0
-260	260.0
-262	262.0
-263	263.0
-265	530.0
-266	266.0
-272	544.0
-273	819.0
-274	274.0
-275	275.0
-277	1108.0
-278	556.0
-280	560.0
-281	562.0
-282	564.0
-283	283.0
-284	284.0
-285	285.0
-286	286.0
-287	287.0
-288	576.0
-289	289.0
-291	291.0
-292	292.0
-296	296.0
-298	894.0
-302	302.0
-305	305.0
-306	306.0
-307	614.0
-308	308.0
-309	618.0
-310	310.0
-311	933.0
-315	315.0
-316	948.0
-317	634.0
-318	954.0
-321	642.0
-322	644.0
-323	323.0
-325	650.0
-327	981.0
-331	662.0
-332	332.0
-333	666.0
-335	335.0
-336	336.0
-338	338.0
-339	339.0
-341	341.0
-342	684.0
-344	688.0
-345	345.0
-348	1740.0
-351	351.0
-353	706.0
-356	356.0
-360	360.0
-362	362.0
-364	364.0
-365	365.0
-366	366.0
-367	734.0
-368	368.0
-369	1107.0
-373	373.0
-374	374.0
-375	375.0
-377	377.0
-378	378.0
-379	379.0
-382	764.0
-384	1152.0
-386	386.0
-389	389.0
-392	392.0
-393	393.0
-394	394.0
-395	790.0
-396	1188.0
-397	794.0
-399	798.0
-400	400.0
-401	2005.0
-402	402.0
-403	1209.0
-404	808.0
-406	1624.0
-407	407.0
-409	1227.0
-411	411.0
-413	826.0
-414	828.0
-417	1251.0
-418	418.0
-419	419.0
-421	421.0
-424	848.0
-427	427.0
-429	858.0
-430	1290.0
-431	1293.0
-432	432.0
-435	435.0
-436	436.0
-437	437.0
-438	1314.0
-439	878.0
-443	443.0
-444	444.0
-446	446.0
-448	448.0
-449	449.0
-452	452.0
-453	453.0
-454	1362.0
-455	455.0
-457	457.0
-458	916.0
-459	918.0
-460	460.0
-462	924.0
-463	926.0
-466	1398.0
-467	467.0
-468	1872.0
-469	2345.0
-470	470.0
-472	472.0
-475	475.0
-477	477.0
-478	956.0
-479	479.0
-480	1440.0
-481	481.0
-482	482.0
-483	483.0
-484	484.0
-485	485.0
-487	487.0
-489	1956.0
-490	490.0
-491	491.0
-492	984.0
-493	493.0
-494	494.0
-495	495.0
-496	496.0
-497	497.0
-498	1494.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby1_map_skew-6-c83b156356c269b6aef263640a4f6b7b
new file mode 100644
index 0000000000000000000000000000000000000000..4953d093489ee79f5dd9f0cbcd91375cc0db53e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-6-c83b156356c269b6aef263640a4f6b7b
@@ -0,0 +1,309 @@
+0	0.0
+2	2.0
+4	4.0
+5	15.0
+8	8.0
+9	9.0
+10	10.0
+11	11.0
+12	24.0
+15	30.0
+17	17.0
+18	36.0
+19	19.0
+20	20.0
+24	48.0
+26	52.0
+27	27.0
+28	28.0
+30	30.0
+33	33.0
+34	34.0
+35	105.0
+37	74.0
+41	41.0
+42	84.0
+43	43.0
+44	44.0
+47	47.0
+51	102.0
+53	53.0
+54	54.0
+57	57.0
+58	116.0
+64	64.0
+65	65.0
+66	66.0
+67	134.0
+69	69.0
+70	210.0
+72	144.0
+74	74.0
+76	152.0
+77	77.0
+78	78.0
+80	80.0
+82	82.0
+83	166.0
+84	168.0
+85	85.0
+86	86.0
+87	87.0
+90	270.0
+92	92.0
+95	190.0
+96	96.0
+97	194.0
+98	196.0
+100	200.0
+103	206.0
+104	208.0
+105	105.0
+111	111.0
+113	226.0
+114	114.0
+116	116.0
+118	236.0
+119	357.0
+120	240.0
+125	250.0
+126	126.0
+128	384.0
+129	258.0
+131	131.0
+133	133.0
+134	268.0
+136	136.0
+137	274.0
+138	552.0
+143	143.0
+145	145.0
+146	292.0
+149	298.0
+150	150.0
+152	304.0
+153	153.0
+155	155.0
+156	156.0
+157	157.0
+158	158.0
+160	160.0
+162	162.0
+163	163.0
+164	328.0
+165	330.0
+166	166.0
+167	501.0
+168	168.0
+169	676.0
+170	170.0
+172	344.0
+174	348.0
+175	350.0
+176	352.0
+177	177.0
+178	178.0
+179	358.0
+180	180.0
+181	181.0
+183	183.0
+186	186.0
+187	561.0
+189	189.0
+190	190.0
+191	382.0
+192	192.0
+193	579.0
+194	194.0
+195	390.0
+196	196.0
+197	394.0
+199	597.0
+200	400.0
+201	201.0
+202	202.0
+203	406.0
+205	410.0
+207	414.0
+208	624.0
+209	418.0
+213	426.0
+214	214.0
+216	432.0
+217	434.0
+218	218.0
+219	438.0
+221	442.0
+222	222.0
+223	446.0
+224	448.0
+226	226.0
+228	228.0
+229	458.0
+230	1150.0
+233	466.0
+235	235.0
+237	474.0
+238	476.0
+239	478.0
+241	241.0
+242	484.0
+244	244.0
+247	247.0
+248	248.0
+249	249.0
+252	252.0
+255	510.0
+256	512.0
+257	257.0
+258	258.0
+260	260.0
+262	262.0
+263	263.0
+265	530.0
+266	266.0
+272	544.0
+273	819.0
+274	274.0
+275	275.0
+277	1108.0
+278	556.0
+280	560.0
+281	562.0
+282	564.0
+283	283.0
+284	284.0
+285	285.0
+286	286.0
+287	287.0
+288	576.0
+289	289.0
+291	291.0
+292	292.0
+296	296.0
+298	894.0
+302	302.0
+305	305.0
+306	306.0
+307	614.0
+308	308.0
+309	618.0
+310	310.0
+311	933.0
+315	315.0
+316	948.0
+317	634.0
+318	954.0
+321	642.0
+322	644.0
+323	323.0
+325	650.0
+327	981.0
+331	662.0
+332	332.0
+333	666.0
+335	335.0
+336	336.0
+338	338.0
+339	339.0
+341	341.0
+342	684.0
+344	688.0
+345	345.0
+348	1740.0
+351	351.0
+353	706.0
+356	356.0
+360	360.0
+362	362.0
+364	364.0
+365	365.0
+366	366.0
+367	734.0
+368	368.0
+369	1107.0
+373	373.0
+374	374.0
+375	375.0
+377	377.0
+378	378.0
+379	379.0
+382	764.0
+384	1152.0
+386	386.0
+389	389.0
+392	392.0
+393	393.0
+394	394.0
+395	790.0
+396	1188.0
+397	794.0
+399	798.0
+400	400.0
+401	2005.0
+402	402.0
+403	1209.0
+404	808.0
+406	1624.0
+407	407.0
+409	1227.0
+411	411.0
+413	826.0
+414	828.0
+417	1251.0
+418	418.0
+419	419.0
+421	421.0
+424	848.0
+427	427.0
+429	858.0
+430	1290.0
+431	1293.0
+432	432.0
+435	435.0
+436	436.0
+437	437.0
+438	1314.0
+439	878.0
+443	443.0
+444	444.0
+446	446.0
+448	448.0
+449	449.0
+452	452.0
+453	453.0
+454	1362.0
+455	455.0
+457	457.0
+458	916.0
+459	918.0
+460	460.0
+462	924.0
+463	926.0
+466	1398.0
+467	467.0
+468	1872.0
+469	2345.0
+470	470.0
+472	472.0
+475	475.0
+477	477.0
+478	956.0
+479	479.0
+480	1440.0
+481	481.0
+482	482.0
+483	483.0
+484	484.0
+485	485.0
+487	487.0
+489	1956.0
+490	490.0
+491	491.0
+492	984.0
+493	493.0
+494	494.0
+495	495.0
+496	496.0
+497	497.0
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 b/sql/hive/src/test/resources/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7
deleted file mode 100644
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7
+++ /dev/null
@@ -1,309 +0,0 @@
-0	0.0
-2	2.0
-4	4.0
-5	15.0
-8	8.0
-9	9.0
-10	10.0
-11	11.0
-12	24.0
-15	30.0
-17	17.0
-18	36.0
-19	19.0
-20	20.0
-24	48.0
-26	52.0
-27	27.0
-28	28.0
-30	30.0
-33	33.0
-34	34.0
-35	105.0
-37	74.0
-41	41.0
-42	84.0
-43	43.0
-44	44.0
-47	47.0
-51	102.0
-53	53.0
-54	54.0
-57	57.0
-58	116.0
-64	64.0
-65	65.0
-66	66.0
-67	134.0
-69	69.0
-70	210.0
-72	144.0
-74	74.0
-76	152.0
-77	77.0
-78	78.0
-80	80.0
-82	82.0
-83	166.0
-84	168.0
-85	85.0
-86	86.0
-87	87.0
-90	270.0
-92	92.0
-95	190.0
-96	96.0
-97	194.0
-98	196.0
-100	200.0
-103	206.0
-104	208.0
-105	105.0
-111	111.0
-113	226.0
-114	114.0
-116	116.0
-118	236.0
-119	357.0
-120	240.0
-125	250.0
-126	126.0
-128	384.0
-129	258.0
-131	131.0
-133	133.0
-134	268.0
-136	136.0
-137	274.0
-138	552.0
-143	143.0
-145	145.0
-146	292.0
-149	298.0
-150	150.0
-152	304.0
-153	153.0
-155	155.0
-156	156.0
-157	157.0
-158	158.0
-160	160.0
-162	162.0
-163	163.0
-164	328.0
-165	330.0
-166	166.0
-167	501.0
-168	168.0
-169	676.0
-170	170.0
-172	344.0
-174	348.0
-175	350.0
-176	352.0
-177	177.0
-178	178.0
-179	358.0
-180	180.0
-181	181.0
-183	183.0
-186	186.0
-187	561.0
-189	189.0
-190	190.0
-191	382.0
-192	192.0
-193	579.0
-194	194.0
-195	390.0
-196	196.0
-197	394.0
-199	597.0
-200	400.0
-201	201.0
-202	202.0
-203	406.0
-205	410.0
-207	414.0
-208	624.0
-209	418.0
-213	426.0
-214	214.0
-216	432.0
-217	434.0
-218	218.0
-219	438.0
-221	442.0
-222	222.0
-223	446.0
-224	448.0
-226	226.0
-228	228.0
-229	458.0
-230	1150.0
-233	466.0
-235	235.0
-237	474.0
-238	476.0
-239	478.0
-241	241.0
-242	484.0
-244	244.0
-247	247.0
-248	248.0
-249	249.0
-252	252.0
-255	510.0
-256	512.0
-257	257.0
-258	258.0
-260	260.0
-262	262.0
-263	263.0
-265	530.0
-266	266.0
-272	544.0
-273	819.0
-274	274.0
-275	275.0
-277	1108.0
-278	556.0
-280	560.0
-281	562.0
-282	564.0
-283	283.0
-284	284.0
-285	285.0
-286	286.0
-287	287.0
-288	576.0
-289	289.0
-291	291.0
-292	292.0
-296	296.0
-298	894.0
-302	302.0
-305	305.0
-306	306.0
-307	614.0
-308	308.0
-309	618.0
-310	310.0
-311	933.0
-315	315.0
-316	948.0
-317	634.0
-318	954.0
-321	642.0
-322	644.0
-323	323.0
-325	650.0
-327	981.0
-331	662.0
-332	332.0
-333	666.0
-335	335.0
-336	336.0
-338	338.0
-339	339.0
-341	341.0
-342	684.0
-344	688.0
-345	345.0
-348	1740.0
-351	351.0
-353	706.0
-356	356.0
-360	360.0
-362	362.0
-364	364.0
-365	365.0
-366	366.0
-367	734.0
-368	368.0
-369	1107.0
-373	373.0
-374	374.0
-375	375.0
-377	377.0
-378	378.0
-379	379.0
-382	764.0
-384	1152.0
-386	386.0
-389	389.0
-392	392.0
-393	393.0
-394	394.0
-395	790.0
-396	1188.0
-397	794.0
-399	798.0
-400	400.0
-401	2005.0
-402	402.0
-403	1209.0
-404	808.0
-406	1624.0
-407	407.0
-409	1227.0
-411	411.0
-413	826.0
-414	828.0
-417	1251.0
-418	418.0
-419	419.0
-421	421.0
-424	848.0
-427	427.0
-429	858.0
-430	1290.0
-431	1293.0
-432	432.0
-435	435.0
-436	436.0
-437	437.0
-438	1314.0
-439	878.0
-443	443.0
-444	444.0
-446	446.0
-448	448.0
-449	449.0
-452	452.0
-453	453.0
-454	1362.0
-455	455.0
-457	457.0
-458	916.0
-459	918.0
-460	460.0
-462	924.0
-463	926.0
-466	1398.0
-467	467.0
-468	1872.0
-469	2345.0
-470	470.0
-472	472.0
-475	475.0
-477	477.0
-478	956.0
-479	479.0
-480	1440.0
-481	481.0
-482	482.0
-483	483.0
-484	484.0
-485	485.0
-487	487.0
-489	1956.0
-490	490.0
-491	491.0
-492	984.0
-493	493.0
-494	494.0
-495	495.0
-496	496.0
-497	497.0
-498	1494.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-6-f7eda18efd187ec4bf4cb135833746cb b/sql/hive/src/test/resources/golden/groupby1_noskew-6-f7eda18efd187ec4bf4cb135833746cb
new file mode 100644
index 0000000000000000000000000000000000000000..4953d093489ee79f5dd9f0cbcd91375cc0db53e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby1_noskew-6-f7eda18efd187ec4bf4cb135833746cb
@@ -0,0 +1,309 @@
+0	0.0
+2	2.0
+4	4.0
+5	15.0
+8	8.0
+9	9.0
+10	10.0
+11	11.0
+12	24.0
+15	30.0
+17	17.0
+18	36.0
+19	19.0
+20	20.0
+24	48.0
+26	52.0
+27	27.0
+28	28.0
+30	30.0
+33	33.0
+34	34.0
+35	105.0
+37	74.0
+41	41.0
+42	84.0
+43	43.0
+44	44.0
+47	47.0
+51	102.0
+53	53.0
+54	54.0
+57	57.0
+58	116.0
+64	64.0
+65	65.0
+66	66.0
+67	134.0
+69	69.0
+70	210.0
+72	144.0
+74	74.0
+76	152.0
+77	77.0
+78	78.0
+80	80.0
+82	82.0
+83	166.0
+84	168.0
+85	85.0
+86	86.0
+87	87.0
+90	270.0
+92	92.0
+95	190.0
+96	96.0
+97	194.0
+98	196.0
+100	200.0
+103	206.0
+104	208.0
+105	105.0
+111	111.0
+113	226.0
+114	114.0
+116	116.0
+118	236.0
+119	357.0
+120	240.0
+125	250.0
+126	126.0
+128	384.0
+129	258.0
+131	131.0
+133	133.0
+134	268.0
+136	136.0
+137	274.0
+138	552.0
+143	143.0
+145	145.0
+146	292.0
+149	298.0
+150	150.0
+152	304.0
+153	153.0
+155	155.0
+156	156.0
+157	157.0
+158	158.0
+160	160.0
+162	162.0
+163	163.0
+164	328.0
+165	330.0
+166	166.0
+167	501.0
+168	168.0
+169	676.0
+170	170.0
+172	344.0
+174	348.0
+175	350.0
+176	352.0
+177	177.0
+178	178.0
+179	358.0
+180	180.0
+181	181.0
+183	183.0
+186	186.0
+187	561.0
+189	189.0
+190	190.0
+191	382.0
+192	192.0
+193	579.0
+194	194.0
+195	390.0
+196	196.0
+197	394.0
+199	597.0
+200	400.0
+201	201.0
+202	202.0
+203	406.0
+205	410.0
+207	414.0
+208	624.0
+209	418.0
+213	426.0
+214	214.0
+216	432.0
+217	434.0
+218	218.0
+219	438.0
+221	442.0
+222	222.0
+223	446.0
+224	448.0
+226	226.0
+228	228.0
+229	458.0
+230	1150.0
+233	466.0
+235	235.0
+237	474.0
+238	476.0
+239	478.0
+241	241.0
+242	484.0
+244	244.0
+247	247.0
+248	248.0
+249	249.0
+252	252.0
+255	510.0
+256	512.0
+257	257.0
+258	258.0
+260	260.0
+262	262.0
+263	263.0
+265	530.0
+266	266.0
+272	544.0
+273	819.0
+274	274.0
+275	275.0
+277	1108.0
+278	556.0
+280	560.0
+281	562.0
+282	564.0
+283	283.0
+284	284.0
+285	285.0
+286	286.0
+287	287.0
+288	576.0
+289	289.0
+291	291.0
+292	292.0
+296	296.0
+298	894.0
+302	302.0
+305	305.0
+306	306.0
+307	614.0
+308	308.0
+309	618.0
+310	310.0
+311	933.0
+315	315.0
+316	948.0
+317	634.0
+318	954.0
+321	642.0
+322	644.0
+323	323.0
+325	650.0
+327	981.0
+331	662.0
+332	332.0
+333	666.0
+335	335.0
+336	336.0
+338	338.0
+339	339.0
+341	341.0
+342	684.0
+344	688.0
+345	345.0
+348	1740.0
+351	351.0
+353	706.0
+356	356.0
+360	360.0
+362	362.0
+364	364.0
+365	365.0
+366	366.0
+367	734.0
+368	368.0
+369	1107.0
+373	373.0
+374	374.0
+375	375.0
+377	377.0
+378	378.0
+379	379.0
+382	764.0
+384	1152.0
+386	386.0
+389	389.0
+392	392.0
+393	393.0
+394	394.0
+395	790.0
+396	1188.0
+397	794.0
+399	798.0
+400	400.0
+401	2005.0
+402	402.0
+403	1209.0
+404	808.0
+406	1624.0
+407	407.0
+409	1227.0
+411	411.0
+413	826.0
+414	828.0
+417	1251.0
+418	418.0
+419	419.0
+421	421.0
+424	848.0
+427	427.0
+429	858.0
+430	1290.0
+431	1293.0
+432	432.0
+435	435.0
+436	436.0
+437	437.0
+438	1314.0
+439	878.0
+443	443.0
+444	444.0
+446	446.0
+448	448.0
+449	449.0
+452	452.0
+453	453.0
+454	1362.0
+455	455.0
+457	457.0
+458	916.0
+459	918.0
+460	460.0
+462	924.0
+463	926.0
+466	1398.0
+467	467.0
+468	1872.0
+469	2345.0
+470	470.0
+472	472.0
+475	475.0
+477	477.0
+478	956.0
+479	479.0
+480	1440.0
+481	481.0
+482	482.0
+483	483.0
+484	484.0
+485	485.0
+487	487.0
+489	1956.0
+490	490.0
+491	491.0
+492	984.0
+493	493.0
+494	494.0
+495	495.0
+496	496.0
+497	497.0
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a
+++ b/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c b/sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c
index 3210a3048ff487ac00cf212272ce53be3de47d30..ba568b8fd6cf266fbf47cb1b5f5f08bb96987e81 100644
--- a/sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c
+++ b/sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c
@@ -7,4 +7,4 @@
 6	5	6398.0
 7	6	7735.0
 8	8	8762.0
-9	7	91047.0
\ No newline at end of file
+9	7	91047.0
diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e b/sql/hive/src/test/resources/golden/groupby2_limit-1-a56d6499aef913e11ef599ac8b4f2a25
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e
rename to sql/hive/src/test/resources/golden/groupby2_limit-1-a56d6499aef913e11ef599ac8b4f2a25
diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 b/sql/hive/src/test/resources/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9
deleted file mode 100644
index 9cf9606d751ac1af6936ba0a62dc3102b77060a2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9
+++ /dev/null
@@ -1,5 +0,0 @@
-0	0.0
-2	2.0
-4	4.0
-5	15.0
-8	8.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-2-e7a95dc27fbfcb10bf92a6db61522b6a b/sql/hive/src/test/resources/golden/groupby2_limit-2-e7a95dc27fbfcb10bf92a6db61522b6a
new file mode 100644
index 0000000000000000000000000000000000000000..f92f60d11d4ef5131ff0e80ba3e176d3a52d3690
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby2_limit-2-e7a95dc27fbfcb10bf92a6db61522b6a
@@ -0,0 +1,5 @@
+0	0.0
+2	2.0
+4	4.0
+5	15.0
+8	8.0
diff --git a/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index 3210a3048ff487ac00cf212272ce53be3de47d30..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,10 +0,0 @@
-0	1	00.0
-1	71	116414.0
-2	69	225571.0
-3	62	332004.0
-4	74	452763.0
-5	6	5397.0
-6	5	6398.0
-7	6	7735.0
-8	8	8762.0
-9	7	91047.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby2_map-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby2_map-6-c83b156356c269b6aef263640a4f6b7b
new file mode 100644
index 0000000000000000000000000000000000000000..ba568b8fd6cf266fbf47cb1b5f5f08bb96987e81
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby2_map-6-c83b156356c269b6aef263640a4f6b7b
@@ -0,0 +1,10 @@
+0	1	00.0
+1	71	116414.0
+2	69	225571.0
+3	62	332004.0
+4	74	452763.0
+5	6	5397.0
+6	5	6398.0
+7	6	7735.0
+8	8	8762.0
+9	7	91047.0
diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index 922f4bfc83e448fc6bbc7ad46f209a1e2748595e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,10 +0,0 @@
-0	1	00.0	0	3
-1	71	116414.0	10044	115
-2	69	225571.0	15780	111
-3	62	332004.0	20119	99
-4	74	452763.0	30965	124
-5	6	5397.0	278	10
-6	5	6398.0	331	6
-7	6	7735.0	447	10
-8	8	8762.0	595	10
-9	7	91047.0	577	12
diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f
index 3210a3048ff487ac00cf212272ce53be3de47d30..ba568b8fd6cf266fbf47cb1b5f5f08bb96987e81 100644
--- a/sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f
@@ -7,4 +7,4 @@
 6	5	6398.0
 7	6	7735.0
 8	8	8762.0
-9	7	91047.0
\ No newline at end of file
+9	7	91047.0
diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-6-232957c90d04497da7fe4e599e0dbb94 b/sql/hive/src/test/resources/golden/groupby2_noskew-6-232957c90d04497da7fe4e599e0dbb94
new file mode 100644
index 0000000000000000000000000000000000000000..ba568b8fd6cf266fbf47cb1b5f5f08bb96987e81
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby2_noskew-6-232957c90d04497da7fe4e599e0dbb94
@@ -0,0 +1,10 @@
+0	1	00.0
+1	71	116414.0
+2	69	225571.0
+3	62	332004.0
+4	74	452763.0
+5	6	5397.0
+6	5	6398.0
+7	6	7735.0
+8	8	8762.0
+9	7	91047.0
diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c b/sql/hive/src/test/resources/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c
deleted file mode 100644
index 3210a3048ff487ac00cf212272ce53be3de47d30..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c
+++ /dev/null
@@ -1,10 +0,0 @@
-0	1	00.0
-1	71	116414.0
-2	69	225571.0
-3	62	332004.0
-4	74	452763.0
-5	6	5397.0
-6	5	6398.0
-7	6	7735.0
-8	8	8762.0
-9	7	91047.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c b/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c
deleted file mode 100644
index 922f4bfc83e448fc6bbc7ad46f209a1e2748595e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c
+++ /dev/null
@@ -1,10 +0,0 @@
-0	1	00.0	0	3
-1	71	116414.0	10044	115
-2	69	225571.0	15780	111
-3	62	332004.0	20119	99
-4	74	452763.0	30965	124
-5	6	5397.0	278	10
-6	5	6398.0	331	6
-7	6	7735.0	447	10
-8	8	8762.0	595	10
-9	7	91047.0	577	12
diff --git a/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f
index f55b5c9eef39f00a9a4d8a58764fdc365b60f081..8b1acc12b635c26f3decadeaa251729d3ce512e9 100644
--- a/sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f
@@ -7,4 +7,4 @@
 6
 7
 8
-9
\ No newline at end of file
+9
diff --git a/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f b/sql/hive/src/test/resources/golden/groupby4_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f
new file mode 100644
index 0000000000000000000000000000000000000000..8b1acc12b635c26f3decadeaa251729d3ce512e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby4_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f
@@ -0,0 +1,10 @@
+0
+1
+2
+3
+4
+5
+6
+7
+8
+9
diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index f55b5c9eef39f00a9a4d8a58764fdc365b60f081..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,10 +0,0 @@
-0
-1
-2
-3
-4
-5
-6
-7
-8
-9
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..4953d093489ee79f5dd9f0cbcd91375cc0db53e9 100644
--- a/sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f
@@ -306,4 +306,4 @@
 495	495.0
 496	496.0
 497	497.0
-498	1494.0
\ No newline at end of file
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f
index 349d8b75d942bc4e9631b074664bc4648e2588be..3975bfc1af512d64ae7595bc70e7f85a001d77bd 100644
--- a/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f
@@ -1 +1 @@
-130091
\ No newline at end of file
+130091
diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f
index 349d8b75d942bc4e9631b074664bc4648e2588be..3975bfc1af512d64ae7595bc70e7f85a001d77bd 100644
--- a/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f
@@ -1 +1 @@
-130091
\ No newline at end of file
+130091
diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,309 +0,0 @@
-0	0.0
-2	2.0
-4	4.0
-5	15.0
-8	8.0
-9	9.0
-10	10.0
-11	11.0
-12	24.0
-15	30.0
-17	17.0
-18	36.0
-19	19.0
-20	20.0
-24	48.0
-26	52.0
-27	27.0
-28	28.0
-30	30.0
-33	33.0
-34	34.0
-35	105.0
-37	74.0
-41	41.0
-42	84.0
-43	43.0
-44	44.0
-47	47.0
-51	102.0
-53	53.0
-54	54.0
-57	57.0
-58	116.0
-64	64.0
-65	65.0
-66	66.0
-67	134.0
-69	69.0
-70	210.0
-72	144.0
-74	74.0
-76	152.0
-77	77.0
-78	78.0
-80	80.0
-82	82.0
-83	166.0
-84	168.0
-85	85.0
-86	86.0
-87	87.0
-90	270.0
-92	92.0
-95	190.0
-96	96.0
-97	194.0
-98	196.0
-100	200.0
-103	206.0
-104	208.0
-105	105.0
-111	111.0
-113	226.0
-114	114.0
-116	116.0
-118	236.0
-119	357.0
-120	240.0
-125	250.0
-126	126.0
-128	384.0
-129	258.0
-131	131.0
-133	133.0
-134	268.0
-136	136.0
-137	274.0
-138	552.0
-143	143.0
-145	145.0
-146	292.0
-149	298.0
-150	150.0
-152	304.0
-153	153.0
-155	155.0
-156	156.0
-157	157.0
-158	158.0
-160	160.0
-162	162.0
-163	163.0
-164	328.0
-165	330.0
-166	166.0
-167	501.0
-168	168.0
-169	676.0
-170	170.0
-172	344.0
-174	348.0
-175	350.0
-176	352.0
-177	177.0
-178	178.0
-179	358.0
-180	180.0
-181	181.0
-183	183.0
-186	186.0
-187	561.0
-189	189.0
-190	190.0
-191	382.0
-192	192.0
-193	579.0
-194	194.0
-195	390.0
-196	196.0
-197	394.0
-199	597.0
-200	400.0
-201	201.0
-202	202.0
-203	406.0
-205	410.0
-207	414.0
-208	624.0
-209	418.0
-213	426.0
-214	214.0
-216	432.0
-217	434.0
-218	218.0
-219	438.0
-221	442.0
-222	222.0
-223	446.0
-224	448.0
-226	226.0
-228	228.0
-229	458.0
-230	1150.0
-233	466.0
-235	235.0
-237	474.0
-238	476.0
-239	478.0
-241	241.0
-242	484.0
-244	244.0
-247	247.0
-248	248.0
-249	249.0
-252	252.0
-255	510.0
-256	512.0
-257	257.0
-258	258.0
-260	260.0
-262	262.0
-263	263.0
-265	530.0
-266	266.0
-272	544.0
-273	819.0
-274	274.0
-275	275.0
-277	1108.0
-278	556.0
-280	560.0
-281	562.0
-282	564.0
-283	283.0
-284	284.0
-285	285.0
-286	286.0
-287	287.0
-288	576.0
-289	289.0
-291	291.0
-292	292.0
-296	296.0
-298	894.0
-302	302.0
-305	305.0
-306	306.0
-307	614.0
-308	308.0
-309	618.0
-310	310.0
-311	933.0
-315	315.0
-316	948.0
-317	634.0
-318	954.0
-321	642.0
-322	644.0
-323	323.0
-325	650.0
-327	981.0
-331	662.0
-332	332.0
-333	666.0
-335	335.0
-336	336.0
-338	338.0
-339	339.0
-341	341.0
-342	684.0
-344	688.0
-345	345.0
-348	1740.0
-351	351.0
-353	706.0
-356	356.0
-360	360.0
-362	362.0
-364	364.0
-365	365.0
-366	366.0
-367	734.0
-368	368.0
-369	1107.0
-373	373.0
-374	374.0
-375	375.0
-377	377.0
-378	378.0
-379	379.0
-382	764.0
-384	1152.0
-386	386.0
-389	389.0
-392	392.0
-393	393.0
-394	394.0
-395	790.0
-396	1188.0
-397	794.0
-399	798.0
-400	400.0
-401	2005.0
-402	402.0
-403	1209.0
-404	808.0
-406	1624.0
-407	407.0
-409	1227.0
-411	411.0
-413	826.0
-414	828.0
-417	1251.0
-418	418.0
-419	419.0
-421	421.0
-424	848.0
-427	427.0
-429	858.0
-430	1290.0
-431	1293.0
-432	432.0
-435	435.0
-436	436.0
-437	437.0
-438	1314.0
-439	878.0
-443	443.0
-444	444.0
-446	446.0
-448	448.0
-449	449.0
-452	452.0
-453	453.0
-454	1362.0
-455	455.0
-457	457.0
-458	916.0
-459	918.0
-460	460.0
-462	924.0
-463	926.0
-466	1398.0
-467	467.0
-468	1872.0
-469	2345.0
-470	470.0
-472	472.0
-475	475.0
-477	477.0
-478	956.0
-479	479.0
-480	1440.0
-481	481.0
-482	482.0
-483	483.0
-484	484.0
-485	485.0
-487	487.0
-489	1956.0
-490	490.0
-491	491.0
-492	984.0
-493	493.0
-494	494.0
-495	495.0
-496	496.0
-497	497.0
-498	1494.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby5_noskew-6-c83b156356c269b6aef263640a4f6b7b
new file mode 100644
index 0000000000000000000000000000000000000000..4953d093489ee79f5dd9f0cbcd91375cc0db53e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby5_noskew-6-c83b156356c269b6aef263640a4f6b7b
@@ -0,0 +1,309 @@
+0	0.0
+2	2.0
+4	4.0
+5	15.0
+8	8.0
+9	9.0
+10	10.0
+11	11.0
+12	24.0
+15	30.0
+17	17.0
+18	36.0
+19	19.0
+20	20.0
+24	48.0
+26	52.0
+27	27.0
+28	28.0
+30	30.0
+33	33.0
+34	34.0
+35	105.0
+37	74.0
+41	41.0
+42	84.0
+43	43.0
+44	44.0
+47	47.0
+51	102.0
+53	53.0
+54	54.0
+57	57.0
+58	116.0
+64	64.0
+65	65.0
+66	66.0
+67	134.0
+69	69.0
+70	210.0
+72	144.0
+74	74.0
+76	152.0
+77	77.0
+78	78.0
+80	80.0
+82	82.0
+83	166.0
+84	168.0
+85	85.0
+86	86.0
+87	87.0
+90	270.0
+92	92.0
+95	190.0
+96	96.0
+97	194.0
+98	196.0
+100	200.0
+103	206.0
+104	208.0
+105	105.0
+111	111.0
+113	226.0
+114	114.0
+116	116.0
+118	236.0
+119	357.0
+120	240.0
+125	250.0
+126	126.0
+128	384.0
+129	258.0
+131	131.0
+133	133.0
+134	268.0
+136	136.0
+137	274.0
+138	552.0
+143	143.0
+145	145.0
+146	292.0
+149	298.0
+150	150.0
+152	304.0
+153	153.0
+155	155.0
+156	156.0
+157	157.0
+158	158.0
+160	160.0
+162	162.0
+163	163.0
+164	328.0
+165	330.0
+166	166.0
+167	501.0
+168	168.0
+169	676.0
+170	170.0
+172	344.0
+174	348.0
+175	350.0
+176	352.0
+177	177.0
+178	178.0
+179	358.0
+180	180.0
+181	181.0
+183	183.0
+186	186.0
+187	561.0
+189	189.0
+190	190.0
+191	382.0
+192	192.0
+193	579.0
+194	194.0
+195	390.0
+196	196.0
+197	394.0
+199	597.0
+200	400.0
+201	201.0
+202	202.0
+203	406.0
+205	410.0
+207	414.0
+208	624.0
+209	418.0
+213	426.0
+214	214.0
+216	432.0
+217	434.0
+218	218.0
+219	438.0
+221	442.0
+222	222.0
+223	446.0
+224	448.0
+226	226.0
+228	228.0
+229	458.0
+230	1150.0
+233	466.0
+235	235.0
+237	474.0
+238	476.0
+239	478.0
+241	241.0
+242	484.0
+244	244.0
+247	247.0
+248	248.0
+249	249.0
+252	252.0
+255	510.0
+256	512.0
+257	257.0
+258	258.0
+260	260.0
+262	262.0
+263	263.0
+265	530.0
+266	266.0
+272	544.0
+273	819.0
+274	274.0
+275	275.0
+277	1108.0
+278	556.0
+280	560.0
+281	562.0
+282	564.0
+283	283.0
+284	284.0
+285	285.0
+286	286.0
+287	287.0
+288	576.0
+289	289.0
+291	291.0
+292	292.0
+296	296.0
+298	894.0
+302	302.0
+305	305.0
+306	306.0
+307	614.0
+308	308.0
+309	618.0
+310	310.0
+311	933.0
+315	315.0
+316	948.0
+317	634.0
+318	954.0
+321	642.0
+322	644.0
+323	323.0
+325	650.0
+327	981.0
+331	662.0
+332	332.0
+333	666.0
+335	335.0
+336	336.0
+338	338.0
+339	339.0
+341	341.0
+342	684.0
+344	688.0
+345	345.0
+348	1740.0
+351	351.0
+353	706.0
+356	356.0
+360	360.0
+362	362.0
+364	364.0
+365	365.0
+366	366.0
+367	734.0
+368	368.0
+369	1107.0
+373	373.0
+374	374.0
+375	375.0
+377	377.0
+378	378.0
+379	379.0
+382	764.0
+384	1152.0
+386	386.0
+389	389.0
+392	392.0
+393	393.0
+394	394.0
+395	790.0
+396	1188.0
+397	794.0
+399	798.0
+400	400.0
+401	2005.0
+402	402.0
+403	1209.0
+404	808.0
+406	1624.0
+407	407.0
+409	1227.0
+411	411.0
+413	826.0
+414	828.0
+417	1251.0
+418	418.0
+419	419.0
+421	421.0
+424	848.0
+427	427.0
+429	858.0
+430	1290.0
+431	1293.0
+432	432.0
+435	435.0
+436	436.0
+437	437.0
+438	1314.0
+439	878.0
+443	443.0
+444	444.0
+446	446.0
+448	448.0
+449	449.0
+452	452.0
+453	453.0
+454	1362.0
+455	455.0
+457	457.0
+458	916.0
+459	918.0
+460	460.0
+462	924.0
+463	926.0
+466	1398.0
+467	467.0
+468	1872.0
+469	2345.0
+470	470.0
+472	472.0
+475	475.0
+477	477.0
+478	956.0
+479	479.0
+480	1440.0
+481	481.0
+482	482.0
+483	483.0
+484	484.0
+485	485.0
+487	487.0
+489	1956.0
+490	490.0
+491	491.0
+492	984.0
+493	493.0
+494	494.0
+495	495.0
+496	496.0
+497	497.0
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f
index f55b5c9eef39f00a9a4d8a58764fdc365b60f081..8b1acc12b635c26f3decadeaa251729d3ce512e9 100644
--- a/sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f
@@ -7,4 +7,4 @@
 6
 7
 8
-9
\ No newline at end of file
+9
diff --git a/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby6_map-6-2d1fb04c7060fafe470e14061b2a5b6f b/sql/hive/src/test/resources/golden/groupby6_map-6-2d1fb04c7060fafe470e14061b2a5b6f
new file mode 100644
index 0000000000000000000000000000000000000000..8b1acc12b635c26f3decadeaa251729d3ce512e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby6_map-6-2d1fb04c7060fafe470e14061b2a5b6f
@@ -0,0 +1,10 @@
+0
+1
+2
+3
+4
+5
+6
+7
+8
+9
diff --git a/sql/hive/src/test/resources/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index f55b5c9eef39f00a9a4d8a58764fdc365b60f081..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,10 +0,0 @@
-0
-1
-2
-3
-4
-5
-6
-7
-8
-9
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-6-2d1fb04c7060fafe470e14061b2a5b6f b/sql/hive/src/test/resources/golden/groupby6_map_skew-6-2d1fb04c7060fafe470e14061b2a5b6f
new file mode 100644
index 0000000000000000000000000000000000000000..8b1acc12b635c26f3decadeaa251729d3ce512e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-6-2d1fb04c7060fafe470e14061b2a5b6f
@@ -0,0 +1,10 @@
+0
+1
+2
+3
+4
+5
+6
+7
+8
+9
diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index f55b5c9eef39f00a9a4d8a58764fdc365b60f081..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,10 +0,0 @@
-0
-1
-2
-3
-4
-5
-6
-7
-8
-9
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f b/sql/hive/src/test/resources/golden/groupby6_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f
new file mode 100644
index 0000000000000000000000000000000000000000..8b1acc12b635c26f3decadeaa251729d3ce512e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby6_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f
@@ -0,0 +1,10 @@
+0
+1
+2
+3
+4
+5
+6
+7
+8
+9
diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index f55b5c9eef39f00a9a4d8a58764fdc365b60f081..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,10 +0,0 @@
-0
-1
-2
-3
-4
-5
-6
-7
-8
-9
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c
+++ b/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403
+++ b/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..4953d093489ee79f5dd9f0cbcd91375cc0db53e9 100644
--- a/sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ b/sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5
@@ -306,4 +306,4 @@
 495	495.0
 496	496.0
 497	497.0
-498	1494.0
\ No newline at end of file
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..4953d093489ee79f5dd9f0cbcd91375cc0db53e9 100644
--- a/sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9
+++ b/sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9
@@ -306,4 +306,4 @@
 495	495.0
 496	496.0
 497	497.0
-498	1494.0
\ No newline at end of file
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e
+++ b/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5
deleted file mode 100644
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ /dev/null
@@ -1,309 +0,0 @@
-0	0.0
-2	2.0
-4	4.0
-5	15.0
-8	8.0
-9	9.0
-10	10.0
-11	11.0
-12	24.0
-15	30.0
-17	17.0
-18	36.0
-19	19.0
-20	20.0
-24	48.0
-26	52.0
-27	27.0
-28	28.0
-30	30.0
-33	33.0
-34	34.0
-35	105.0
-37	74.0
-41	41.0
-42	84.0
-43	43.0
-44	44.0
-47	47.0
-51	102.0
-53	53.0
-54	54.0
-57	57.0
-58	116.0
-64	64.0
-65	65.0
-66	66.0
-67	134.0
-69	69.0
-70	210.0
-72	144.0
-74	74.0
-76	152.0
-77	77.0
-78	78.0
-80	80.0
-82	82.0
-83	166.0
-84	168.0
-85	85.0
-86	86.0
-87	87.0
-90	270.0
-92	92.0
-95	190.0
-96	96.0
-97	194.0
-98	196.0
-100	200.0
-103	206.0
-104	208.0
-105	105.0
-111	111.0
-113	226.0
-114	114.0
-116	116.0
-118	236.0
-119	357.0
-120	240.0
-125	250.0
-126	126.0
-128	384.0
-129	258.0
-131	131.0
-133	133.0
-134	268.0
-136	136.0
-137	274.0
-138	552.0
-143	143.0
-145	145.0
-146	292.0
-149	298.0
-150	150.0
-152	304.0
-153	153.0
-155	155.0
-156	156.0
-157	157.0
-158	158.0
-160	160.0
-162	162.0
-163	163.0
-164	328.0
-165	330.0
-166	166.0
-167	501.0
-168	168.0
-169	676.0
-170	170.0
-172	344.0
-174	348.0
-175	350.0
-176	352.0
-177	177.0
-178	178.0
-179	358.0
-180	180.0
-181	181.0
-183	183.0
-186	186.0
-187	561.0
-189	189.0
-190	190.0
-191	382.0
-192	192.0
-193	579.0
-194	194.0
-195	390.0
-196	196.0
-197	394.0
-199	597.0
-200	400.0
-201	201.0
-202	202.0
-203	406.0
-205	410.0
-207	414.0
-208	624.0
-209	418.0
-213	426.0
-214	214.0
-216	432.0
-217	434.0
-218	218.0
-219	438.0
-221	442.0
-222	222.0
-223	446.0
-224	448.0
-226	226.0
-228	228.0
-229	458.0
-230	1150.0
-233	466.0
-235	235.0
-237	474.0
-238	476.0
-239	478.0
-241	241.0
-242	484.0
-244	244.0
-247	247.0
-248	248.0
-249	249.0
-252	252.0
-255	510.0
-256	512.0
-257	257.0
-258	258.0
-260	260.0
-262	262.0
-263	263.0
-265	530.0
-266	266.0
-272	544.0
-273	819.0
-274	274.0
-275	275.0
-277	1108.0
-278	556.0
-280	560.0
-281	562.0
-282	564.0
-283	283.0
-284	284.0
-285	285.0
-286	286.0
-287	287.0
-288	576.0
-289	289.0
-291	291.0
-292	292.0
-296	296.0
-298	894.0
-302	302.0
-305	305.0
-306	306.0
-307	614.0
-308	308.0
-309	618.0
-310	310.0
-311	933.0
-315	315.0
-316	948.0
-317	634.0
-318	954.0
-321	642.0
-322	644.0
-323	323.0
-325	650.0
-327	981.0
-331	662.0
-332	332.0
-333	666.0
-335	335.0
-336	336.0
-338	338.0
-339	339.0
-341	341.0
-342	684.0
-344	688.0
-345	345.0
-348	1740.0
-351	351.0
-353	706.0
-356	356.0
-360	360.0
-362	362.0
-364	364.0
-365	365.0
-366	366.0
-367	734.0
-368	368.0
-369	1107.0
-373	373.0
-374	374.0
-375	375.0
-377	377.0
-378	378.0
-379	379.0
-382	764.0
-384	1152.0
-386	386.0
-389	389.0
-392	392.0
-393	393.0
-394	394.0
-395	790.0
-396	1188.0
-397	794.0
-399	798.0
-400	400.0
-401	2005.0
-402	402.0
-403	1209.0
-404	808.0
-406	1624.0
-407	407.0
-409	1227.0
-411	411.0
-413	826.0
-414	828.0
-417	1251.0
-418	418.0
-419	419.0
-421	421.0
-424	848.0
-427	427.0
-429	858.0
-430	1290.0
-431	1293.0
-432	432.0
-435	435.0
-436	436.0
-437	437.0
-438	1314.0
-439	878.0
-443	443.0
-444	444.0
-446	446.0
-448	448.0
-449	449.0
-452	452.0
-453	453.0
-454	1362.0
-455	455.0
-457	457.0
-458	916.0
-459	918.0
-460	460.0
-462	924.0
-463	926.0
-466	1398.0
-467	467.0
-468	1872.0
-469	2345.0
-470	470.0
-472	472.0
-475	475.0
-477	477.0
-478	956.0
-479	479.0
-480	1440.0
-481	481.0
-482	482.0
-483	483.0
-484	484.0
-485	485.0
-487	487.0
-489	1956.0
-490	490.0
-491	491.0
-492	984.0
-493	493.0
-494	494.0
-495	495.0
-496	496.0
-497	497.0
-498	1494.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby7_map-10-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby7_map-10-c1a8cf4ef8060a4703b0affe40496169
new file mode 100644
index 0000000000000000000000000000000000000000..4953d093489ee79f5dd9f0cbcd91375cc0db53e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby7_map-10-c1a8cf4ef8060a4703b0affe40496169
@@ -0,0 +1,309 @@
+0	0.0
+2	2.0
+4	4.0
+5	15.0
+8	8.0
+9	9.0
+10	10.0
+11	11.0
+12	24.0
+15	30.0
+17	17.0
+18	36.0
+19	19.0
+20	20.0
+24	48.0
+26	52.0
+27	27.0
+28	28.0
+30	30.0
+33	33.0
+34	34.0
+35	105.0
+37	74.0
+41	41.0
+42	84.0
+43	43.0
+44	44.0
+47	47.0
+51	102.0
+53	53.0
+54	54.0
+57	57.0
+58	116.0
+64	64.0
+65	65.0
+66	66.0
+67	134.0
+69	69.0
+70	210.0
+72	144.0
+74	74.0
+76	152.0
+77	77.0
+78	78.0
+80	80.0
+82	82.0
+83	166.0
+84	168.0
+85	85.0
+86	86.0
+87	87.0
+90	270.0
+92	92.0
+95	190.0
+96	96.0
+97	194.0
+98	196.0
+100	200.0
+103	206.0
+104	208.0
+105	105.0
+111	111.0
+113	226.0
+114	114.0
+116	116.0
+118	236.0
+119	357.0
+120	240.0
+125	250.0
+126	126.0
+128	384.0
+129	258.0
+131	131.0
+133	133.0
+134	268.0
+136	136.0
+137	274.0
+138	552.0
+143	143.0
+145	145.0
+146	292.0
+149	298.0
+150	150.0
+152	304.0
+153	153.0
+155	155.0
+156	156.0
+157	157.0
+158	158.0
+160	160.0
+162	162.0
+163	163.0
+164	328.0
+165	330.0
+166	166.0
+167	501.0
+168	168.0
+169	676.0
+170	170.0
+172	344.0
+174	348.0
+175	350.0
+176	352.0
+177	177.0
+178	178.0
+179	358.0
+180	180.0
+181	181.0
+183	183.0
+186	186.0
+187	561.0
+189	189.0
+190	190.0
+191	382.0
+192	192.0
+193	579.0
+194	194.0
+195	390.0
+196	196.0
+197	394.0
+199	597.0
+200	400.0
+201	201.0
+202	202.0
+203	406.0
+205	410.0
+207	414.0
+208	624.0
+209	418.0
+213	426.0
+214	214.0
+216	432.0
+217	434.0
+218	218.0
+219	438.0
+221	442.0
+222	222.0
+223	446.0
+224	448.0
+226	226.0
+228	228.0
+229	458.0
+230	1150.0
+233	466.0
+235	235.0
+237	474.0
+238	476.0
+239	478.0
+241	241.0
+242	484.0
+244	244.0
+247	247.0
+248	248.0
+249	249.0
+252	252.0
+255	510.0
+256	512.0
+257	257.0
+258	258.0
+260	260.0
+262	262.0
+263	263.0
+265	530.0
+266	266.0
+272	544.0
+273	819.0
+274	274.0
+275	275.0
+277	1108.0
+278	556.0
+280	560.0
+281	562.0
+282	564.0
+283	283.0
+284	284.0
+285	285.0
+286	286.0
+287	287.0
+288	576.0
+289	289.0
+291	291.0
+292	292.0
+296	296.0
+298	894.0
+302	302.0
+305	305.0
+306	306.0
+307	614.0
+308	308.0
+309	618.0
+310	310.0
+311	933.0
+315	315.0
+316	948.0
+317	634.0
+318	954.0
+321	642.0
+322	644.0
+323	323.0
+325	650.0
+327	981.0
+331	662.0
+332	332.0
+333	666.0
+335	335.0
+336	336.0
+338	338.0
+339	339.0
+341	341.0
+342	684.0
+344	688.0
+345	345.0
+348	1740.0
+351	351.0
+353	706.0
+356	356.0
+360	360.0
+362	362.0
+364	364.0
+365	365.0
+366	366.0
+367	734.0
+368	368.0
+369	1107.0
+373	373.0
+374	374.0
+375	375.0
+377	377.0
+378	378.0
+379	379.0
+382	764.0
+384	1152.0
+386	386.0
+389	389.0
+392	392.0
+393	393.0
+394	394.0
+395	790.0
+396	1188.0
+397	794.0
+399	798.0
+400	400.0
+401	2005.0
+402	402.0
+403	1209.0
+404	808.0
+406	1624.0
+407	407.0
+409	1227.0
+411	411.0
+413	826.0
+414	828.0
+417	1251.0
+418	418.0
+419	419.0
+421	421.0
+424	848.0
+427	427.0
+429	858.0
+430	1290.0
+431	1293.0
+432	432.0
+435	435.0
+436	436.0
+437	437.0
+438	1314.0
+439	878.0
+443	443.0
+444	444.0
+446	446.0
+448	448.0
+449	449.0
+452	452.0
+453	453.0
+454	1362.0
+455	455.0
+457	457.0
+458	916.0
+459	918.0
+460	460.0
+462	924.0
+463	926.0
+466	1398.0
+467	467.0
+468	1872.0
+469	2345.0
+470	470.0
+472	472.0
+475	475.0
+477	477.0
+478	956.0
+479	479.0
+480	1440.0
+481	481.0
+482	482.0
+483	483.0
+484	484.0
+485	485.0
+487	487.0
+489	1956.0
+490	490.0
+491	491.0
+492	984.0
+493	493.0
+494	494.0
+495	495.0
+496	496.0
+497	497.0
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9
deleted file mode 100644
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9
+++ /dev/null
@@ -1,309 +0,0 @@
-0	0.0
-2	2.0
-4	4.0
-5	15.0
-8	8.0
-9	9.0
-10	10.0
-11	11.0
-12	24.0
-15	30.0
-17	17.0
-18	36.0
-19	19.0
-20	20.0
-24	48.0
-26	52.0
-27	27.0
-28	28.0
-30	30.0
-33	33.0
-34	34.0
-35	105.0
-37	74.0
-41	41.0
-42	84.0
-43	43.0
-44	44.0
-47	47.0
-51	102.0
-53	53.0
-54	54.0
-57	57.0
-58	116.0
-64	64.0
-65	65.0
-66	66.0
-67	134.0
-69	69.0
-70	210.0
-72	144.0
-74	74.0
-76	152.0
-77	77.0
-78	78.0
-80	80.0
-82	82.0
-83	166.0
-84	168.0
-85	85.0
-86	86.0
-87	87.0
-90	270.0
-92	92.0
-95	190.0
-96	96.0
-97	194.0
-98	196.0
-100	200.0
-103	206.0
-104	208.0
-105	105.0
-111	111.0
-113	226.0
-114	114.0
-116	116.0
-118	236.0
-119	357.0
-120	240.0
-125	250.0
-126	126.0
-128	384.0
-129	258.0
-131	131.0
-133	133.0
-134	268.0
-136	136.0
-137	274.0
-138	552.0
-143	143.0
-145	145.0
-146	292.0
-149	298.0
-150	150.0
-152	304.0
-153	153.0
-155	155.0
-156	156.0
-157	157.0
-158	158.0
-160	160.0
-162	162.0
-163	163.0
-164	328.0
-165	330.0
-166	166.0
-167	501.0
-168	168.0
-169	676.0
-170	170.0
-172	344.0
-174	348.0
-175	350.0
-176	352.0
-177	177.0
-178	178.0
-179	358.0
-180	180.0
-181	181.0
-183	183.0
-186	186.0
-187	561.0
-189	189.0
-190	190.0
-191	382.0
-192	192.0
-193	579.0
-194	194.0
-195	390.0
-196	196.0
-197	394.0
-199	597.0
-200	400.0
-201	201.0
-202	202.0
-203	406.0
-205	410.0
-207	414.0
-208	624.0
-209	418.0
-213	426.0
-214	214.0
-216	432.0
-217	434.0
-218	218.0
-219	438.0
-221	442.0
-222	222.0
-223	446.0
-224	448.0
-226	226.0
-228	228.0
-229	458.0
-230	1150.0
-233	466.0
-235	235.0
-237	474.0
-238	476.0
-239	478.0
-241	241.0
-242	484.0
-244	244.0
-247	247.0
-248	248.0
-249	249.0
-252	252.0
-255	510.0
-256	512.0
-257	257.0
-258	258.0
-260	260.0
-262	262.0
-263	263.0
-265	530.0
-266	266.0
-272	544.0
-273	819.0
-274	274.0
-275	275.0
-277	1108.0
-278	556.0
-280	560.0
-281	562.0
-282	564.0
-283	283.0
-284	284.0
-285	285.0
-286	286.0
-287	287.0
-288	576.0
-289	289.0
-291	291.0
-292	292.0
-296	296.0
-298	894.0
-302	302.0
-305	305.0
-306	306.0
-307	614.0
-308	308.0
-309	618.0
-310	310.0
-311	933.0
-315	315.0
-316	948.0
-317	634.0
-318	954.0
-321	642.0
-322	644.0
-323	323.0
-325	650.0
-327	981.0
-331	662.0
-332	332.0
-333	666.0
-335	335.0
-336	336.0
-338	338.0
-339	339.0
-341	341.0
-342	684.0
-344	688.0
-345	345.0
-348	1740.0
-351	351.0
-353	706.0
-356	356.0
-360	360.0
-362	362.0
-364	364.0
-365	365.0
-366	366.0
-367	734.0
-368	368.0
-369	1107.0
-373	373.0
-374	374.0
-375	375.0
-377	377.0
-378	378.0
-379	379.0
-382	764.0
-384	1152.0
-386	386.0
-389	389.0
-392	392.0
-393	393.0
-394	394.0
-395	790.0
-396	1188.0
-397	794.0
-399	798.0
-400	400.0
-401	2005.0
-402	402.0
-403	1209.0
-404	808.0
-406	1624.0
-407	407.0
-409	1227.0
-411	411.0
-413	826.0
-414	828.0
-417	1251.0
-418	418.0
-419	419.0
-421	421.0
-424	848.0
-427	427.0
-429	858.0
-430	1290.0
-431	1293.0
-432	432.0
-435	435.0
-436	436.0
-437	437.0
-438	1314.0
-439	878.0
-443	443.0
-444	444.0
-446	446.0
-448	448.0
-449	449.0
-452	452.0
-453	453.0
-454	1362.0
-455	455.0
-457	457.0
-458	916.0
-459	918.0
-460	460.0
-462	924.0
-463	926.0
-466	1398.0
-467	467.0
-468	1872.0
-469	2345.0
-470	470.0
-472	472.0
-475	475.0
-477	477.0
-478	956.0
-479	479.0
-480	1440.0
-481	481.0
-482	482.0
-483	483.0
-484	484.0
-485	485.0
-487	487.0
-489	1956.0
-490	490.0
-491	491.0
-492	984.0
-493	493.0
-494	494.0
-495	495.0
-496	496.0
-497	497.0
-498	1494.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby7_map-11-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby7_map-11-6c26c5d39c4bdb61728defa9b44bfb52
new file mode 100644
index 0000000000000000000000000000000000000000..4953d093489ee79f5dd9f0cbcd91375cc0db53e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby7_map-11-6c26c5d39c4bdb61728defa9b44bfb52
@@ -0,0 +1,309 @@
+0	0.0
+2	2.0
+4	4.0
+5	15.0
+8	8.0
+9	9.0
+10	10.0
+11	11.0
+12	24.0
+15	30.0
+17	17.0
+18	36.0
+19	19.0
+20	20.0
+24	48.0
+26	52.0
+27	27.0
+28	28.0
+30	30.0
+33	33.0
+34	34.0
+35	105.0
+37	74.0
+41	41.0
+42	84.0
+43	43.0
+44	44.0
+47	47.0
+51	102.0
+53	53.0
+54	54.0
+57	57.0
+58	116.0
+64	64.0
+65	65.0
+66	66.0
+67	134.0
+69	69.0
+70	210.0
+72	144.0
+74	74.0
+76	152.0
+77	77.0
+78	78.0
+80	80.0
+82	82.0
+83	166.0
+84	168.0
+85	85.0
+86	86.0
+87	87.0
+90	270.0
+92	92.0
+95	190.0
+96	96.0
+97	194.0
+98	196.0
+100	200.0
+103	206.0
+104	208.0
+105	105.0
+111	111.0
+113	226.0
+114	114.0
+116	116.0
+118	236.0
+119	357.0
+120	240.0
+125	250.0
+126	126.0
+128	384.0
+129	258.0
+131	131.0
+133	133.0
+134	268.0
+136	136.0
+137	274.0
+138	552.0
+143	143.0
+145	145.0
+146	292.0
+149	298.0
+150	150.0
+152	304.0
+153	153.0
+155	155.0
+156	156.0
+157	157.0
+158	158.0
+160	160.0
+162	162.0
+163	163.0
+164	328.0
+165	330.0
+166	166.0
+167	501.0
+168	168.0
+169	676.0
+170	170.0
+172	344.0
+174	348.0
+175	350.0
+176	352.0
+177	177.0
+178	178.0
+179	358.0
+180	180.0
+181	181.0
+183	183.0
+186	186.0
+187	561.0
+189	189.0
+190	190.0
+191	382.0
+192	192.0
+193	579.0
+194	194.0
+195	390.0
+196	196.0
+197	394.0
+199	597.0
+200	400.0
+201	201.0
+202	202.0
+203	406.0
+205	410.0
+207	414.0
+208	624.0
+209	418.0
+213	426.0
+214	214.0
+216	432.0
+217	434.0
+218	218.0
+219	438.0
+221	442.0
+222	222.0
+223	446.0
+224	448.0
+226	226.0
+228	228.0
+229	458.0
+230	1150.0
+233	466.0
+235	235.0
+237	474.0
+238	476.0
+239	478.0
+241	241.0
+242	484.0
+244	244.0
+247	247.0
+248	248.0
+249	249.0
+252	252.0
+255	510.0
+256	512.0
+257	257.0
+258	258.0
+260	260.0
+262	262.0
+263	263.0
+265	530.0
+266	266.0
+272	544.0
+273	819.0
+274	274.0
+275	275.0
+277	1108.0
+278	556.0
+280	560.0
+281	562.0
+282	564.0
+283	283.0
+284	284.0
+285	285.0
+286	286.0
+287	287.0
+288	576.0
+289	289.0
+291	291.0
+292	292.0
+296	296.0
+298	894.0
+302	302.0
+305	305.0
+306	306.0
+307	614.0
+308	308.0
+309	618.0
+310	310.0
+311	933.0
+315	315.0
+316	948.0
+317	634.0
+318	954.0
+321	642.0
+322	644.0
+323	323.0
+325	650.0
+327	981.0
+331	662.0
+332	332.0
+333	666.0
+335	335.0
+336	336.0
+338	338.0
+339	339.0
+341	341.0
+342	684.0
+344	688.0
+345	345.0
+348	1740.0
+351	351.0
+353	706.0
+356	356.0
+360	360.0
+362	362.0
+364	364.0
+365	365.0
+366	366.0
+367	734.0
+368	368.0
+369	1107.0
+373	373.0
+374	374.0
+375	375.0
+377	377.0
+378	378.0
+379	379.0
+382	764.0
+384	1152.0
+386	386.0
+389	389.0
+392	392.0
+393	393.0
+394	394.0
+395	790.0
+396	1188.0
+397	794.0
+399	798.0
+400	400.0
+401	2005.0
+402	402.0
+403	1209.0
+404	808.0
+406	1624.0
+407	407.0
+409	1227.0
+411	411.0
+413	826.0
+414	828.0
+417	1251.0
+418	418.0
+419	419.0
+421	421.0
+424	848.0
+427	427.0
+429	858.0
+430	1290.0
+431	1293.0
+432	432.0
+435	435.0
+436	436.0
+437	437.0
+438	1314.0
+439	878.0
+443	443.0
+444	444.0
+446	446.0
+448	448.0
+449	449.0
+452	452.0
+453	453.0
+454	1362.0
+455	455.0
+457	457.0
+458	916.0
+459	918.0
+460	460.0
+462	924.0
+463	926.0
+466	1398.0
+467	467.0
+468	1872.0
+469	2345.0
+470	470.0
+472	472.0
+475	475.0
+477	477.0
+478	956.0
+479	479.0
+480	1440.0
+481	481.0
+482	482.0
+483	483.0
+484	484.0
+485	485.0
+487	487.0
+489	1956.0
+490	490.0
+491	491.0
+492	984.0
+493	493.0
+494	494.0
+495	495.0
+496	496.0
+497	497.0
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c
+++ b/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403
+++ b/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9
deleted file mode 100644
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9
+++ /dev/null
@@ -1,309 +0,0 @@
-0	0.0
-2	2.0
-4	4.0
-5	15.0
-8	8.0
-9	9.0
-10	10.0
-11	11.0
-12	24.0
-15	30.0
-17	17.0
-18	36.0
-19	19.0
-20	20.0
-24	48.0
-26	52.0
-27	27.0
-28	28.0
-30	30.0
-33	33.0
-34	34.0
-35	105.0
-37	74.0
-41	41.0
-42	84.0
-43	43.0
-44	44.0
-47	47.0
-51	102.0
-53	53.0
-54	54.0
-57	57.0
-58	116.0
-64	64.0
-65	65.0
-66	66.0
-67	134.0
-69	69.0
-70	210.0
-72	144.0
-74	74.0
-76	152.0
-77	77.0
-78	78.0
-80	80.0
-82	82.0
-83	166.0
-84	168.0
-85	85.0
-86	86.0
-87	87.0
-90	270.0
-92	92.0
-95	190.0
-96	96.0
-97	194.0
-98	196.0
-100	200.0
-103	206.0
-104	208.0
-105	105.0
-111	111.0
-113	226.0
-114	114.0
-116	116.0
-118	236.0
-119	357.0
-120	240.0
-125	250.0
-126	126.0
-128	384.0
-129	258.0
-131	131.0
-133	133.0
-134	268.0
-136	136.0
-137	274.0
-138	552.0
-143	143.0
-145	145.0
-146	292.0
-149	298.0
-150	150.0
-152	304.0
-153	153.0
-155	155.0
-156	156.0
-157	157.0
-158	158.0
-160	160.0
-162	162.0
-163	163.0
-164	328.0
-165	330.0
-166	166.0
-167	501.0
-168	168.0
-169	676.0
-170	170.0
-172	344.0
-174	348.0
-175	350.0
-176	352.0
-177	177.0
-178	178.0
-179	358.0
-180	180.0
-181	181.0
-183	183.0
-186	186.0
-187	561.0
-189	189.0
-190	190.0
-191	382.0
-192	192.0
-193	579.0
-194	194.0
-195	390.0
-196	196.0
-197	394.0
-199	597.0
-200	400.0
-201	201.0
-202	202.0
-203	406.0
-205	410.0
-207	414.0
-208	624.0
-209	418.0
-213	426.0
-214	214.0
-216	432.0
-217	434.0
-218	218.0
-219	438.0
-221	442.0
-222	222.0
-223	446.0
-224	448.0
-226	226.0
-228	228.0
-229	458.0
-230	1150.0
-233	466.0
-235	235.0
-237	474.0
-238	476.0
-239	478.0
-241	241.0
-242	484.0
-244	244.0
-247	247.0
-248	248.0
-249	249.0
-252	252.0
-255	510.0
-256	512.0
-257	257.0
-258	258.0
-260	260.0
-262	262.0
-263	263.0
-265	530.0
-266	266.0
-272	544.0
-273	819.0
-274	274.0
-275	275.0
-277	1108.0
-278	556.0
-280	560.0
-281	562.0
-282	564.0
-283	283.0
-284	284.0
-285	285.0
-286	286.0
-287	287.0
-288	576.0
-289	289.0
-291	291.0
-292	292.0
-296	296.0
-298	894.0
-302	302.0
-305	305.0
-306	306.0
-307	614.0
-308	308.0
-309	618.0
-310	310.0
-311	933.0
-315	315.0
-316	948.0
-317	634.0
-318	954.0
-321	642.0
-322	644.0
-323	323.0
-325	650.0
-327	981.0
-331	662.0
-332	332.0
-333	666.0
-335	335.0
-336	336.0
-338	338.0
-339	339.0
-341	341.0
-342	684.0
-344	688.0
-345	345.0
-348	1740.0
-351	351.0
-353	706.0
-356	356.0
-360	360.0
-362	362.0
-364	364.0
-365	365.0
-366	366.0
-367	734.0
-368	368.0
-369	1107.0
-373	373.0
-374	374.0
-375	375.0
-377	377.0
-378	378.0
-379	379.0
-382	764.0
-384	1152.0
-386	386.0
-389	389.0
-392	392.0
-393	393.0
-394	394.0
-395	790.0
-396	1188.0
-397	794.0
-399	798.0
-400	400.0
-401	2005.0
-402	402.0
-403	1209.0
-404	808.0
-406	1624.0
-407	407.0
-409	1227.0
-411	411.0
-413	826.0
-414	828.0
-417	1251.0
-418	418.0
-419	419.0
-421	421.0
-424	848.0
-427	427.0
-429	858.0
-430	1290.0
-431	1293.0
-432	432.0
-435	435.0
-436	436.0
-437	437.0
-438	1314.0
-439	878.0
-443	443.0
-444	444.0
-446	446.0
-448	448.0
-449	449.0
-452	452.0
-453	453.0
-454	1362.0
-455	455.0
-457	457.0
-458	916.0
-459	918.0
-460	460.0
-462	924.0
-463	926.0
-466	1398.0
-467	467.0
-468	1872.0
-469	2345.0
-470	470.0
-472	472.0
-475	475.0
-477	477.0
-478	956.0
-479	479.0
-480	1440.0
-481	481.0
-482	482.0
-483	483.0
-484	484.0
-485	485.0
-487	487.0
-489	1956.0
-490	490.0
-491	491.0
-492	984.0
-493	493.0
-494	494.0
-495	495.0
-496	496.0
-497	497.0
-498	1494.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-6c26c5d39c4bdb61728defa9b44bfb52
new file mode 100644
index 0000000000000000000000000000000000000000..4953d093489ee79f5dd9f0cbcd91375cc0db53e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-6c26c5d39c4bdb61728defa9b44bfb52
@@ -0,0 +1,309 @@
+0	0.0
+2	2.0
+4	4.0
+5	15.0
+8	8.0
+9	9.0
+10	10.0
+11	11.0
+12	24.0
+15	30.0
+17	17.0
+18	36.0
+19	19.0
+20	20.0
+24	48.0
+26	52.0
+27	27.0
+28	28.0
+30	30.0
+33	33.0
+34	34.0
+35	105.0
+37	74.0
+41	41.0
+42	84.0
+43	43.0
+44	44.0
+47	47.0
+51	102.0
+53	53.0
+54	54.0
+57	57.0
+58	116.0
+64	64.0
+65	65.0
+66	66.0
+67	134.0
+69	69.0
+70	210.0
+72	144.0
+74	74.0
+76	152.0
+77	77.0
+78	78.0
+80	80.0
+82	82.0
+83	166.0
+84	168.0
+85	85.0
+86	86.0
+87	87.0
+90	270.0
+92	92.0
+95	190.0
+96	96.0
+97	194.0
+98	196.0
+100	200.0
+103	206.0
+104	208.0
+105	105.0
+111	111.0
+113	226.0
+114	114.0
+116	116.0
+118	236.0
+119	357.0
+120	240.0
+125	250.0
+126	126.0
+128	384.0
+129	258.0
+131	131.0
+133	133.0
+134	268.0
+136	136.0
+137	274.0
+138	552.0
+143	143.0
+145	145.0
+146	292.0
+149	298.0
+150	150.0
+152	304.0
+153	153.0
+155	155.0
+156	156.0
+157	157.0
+158	158.0
+160	160.0
+162	162.0
+163	163.0
+164	328.0
+165	330.0
+166	166.0
+167	501.0
+168	168.0
+169	676.0
+170	170.0
+172	344.0
+174	348.0
+175	350.0
+176	352.0
+177	177.0
+178	178.0
+179	358.0
+180	180.0
+181	181.0
+183	183.0
+186	186.0
+187	561.0
+189	189.0
+190	190.0
+191	382.0
+192	192.0
+193	579.0
+194	194.0
+195	390.0
+196	196.0
+197	394.0
+199	597.0
+200	400.0
+201	201.0
+202	202.0
+203	406.0
+205	410.0
+207	414.0
+208	624.0
+209	418.0
+213	426.0
+214	214.0
+216	432.0
+217	434.0
+218	218.0
+219	438.0
+221	442.0
+222	222.0
+223	446.0
+224	448.0
+226	226.0
+228	228.0
+229	458.0
+230	1150.0
+233	466.0
+235	235.0
+237	474.0
+238	476.0
+239	478.0
+241	241.0
+242	484.0
+244	244.0
+247	247.0
+248	248.0
+249	249.0
+252	252.0
+255	510.0
+256	512.0
+257	257.0
+258	258.0
+260	260.0
+262	262.0
+263	263.0
+265	530.0
+266	266.0
+272	544.0
+273	819.0
+274	274.0
+275	275.0
+277	1108.0
+278	556.0
+280	560.0
+281	562.0
+282	564.0
+283	283.0
+284	284.0
+285	285.0
+286	286.0
+287	287.0
+288	576.0
+289	289.0
+291	291.0
+292	292.0
+296	296.0
+298	894.0
+302	302.0
+305	305.0
+306	306.0
+307	614.0
+308	308.0
+309	618.0
+310	310.0
+311	933.0
+315	315.0
+316	948.0
+317	634.0
+318	954.0
+321	642.0
+322	644.0
+323	323.0
+325	650.0
+327	981.0
+331	662.0
+332	332.0
+333	666.0
+335	335.0
+336	336.0
+338	338.0
+339	339.0
+341	341.0
+342	684.0
+344	688.0
+345	345.0
+348	1740.0
+351	351.0
+353	706.0
+356	356.0
+360	360.0
+362	362.0
+364	364.0
+365	365.0
+366	366.0
+367	734.0
+368	368.0
+369	1107.0
+373	373.0
+374	374.0
+375	375.0
+377	377.0
+378	378.0
+379	379.0
+382	764.0
+384	1152.0
+386	386.0
+389	389.0
+392	392.0
+393	393.0
+394	394.0
+395	790.0
+396	1188.0
+397	794.0
+399	798.0
+400	400.0
+401	2005.0
+402	402.0
+403	1209.0
+404	808.0
+406	1624.0
+407	407.0
+409	1227.0
+411	411.0
+413	826.0
+414	828.0
+417	1251.0
+418	418.0
+419	419.0
+421	421.0
+424	848.0
+427	427.0
+429	858.0
+430	1290.0
+431	1293.0
+432	432.0
+435	435.0
+436	436.0
+437	437.0
+438	1314.0
+439	878.0
+443	443.0
+444	444.0
+446	446.0
+448	448.0
+449	449.0
+452	452.0
+453	453.0
+454	1362.0
+455	455.0
+457	457.0
+458	916.0
+459	918.0
+460	460.0
+462	924.0
+463	926.0
+466	1398.0
+467	467.0
+468	1872.0
+469	2345.0
+470	470.0
+472	472.0
+475	475.0
+477	477.0
+478	956.0
+479	479.0
+480	1440.0
+481	481.0
+482	482.0
+483	483.0
+484	484.0
+485	485.0
+487	487.0
+489	1956.0
+490	490.0
+491	491.0
+492	984.0
+493	493.0
+494	494.0
+495	495.0
+496	496.0
+497	497.0
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c
+++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403
+++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5
deleted file mode 100644
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ /dev/null
@@ -1,309 +0,0 @@
-0	0.0
-2	2.0
-4	4.0
-5	15.0
-8	8.0
-9	9.0
-10	10.0
-11	11.0
-12	24.0
-15	30.0
-17	17.0
-18	36.0
-19	19.0
-20	20.0
-24	48.0
-26	52.0
-27	27.0
-28	28.0
-30	30.0
-33	33.0
-34	34.0
-35	105.0
-37	74.0
-41	41.0
-42	84.0
-43	43.0
-44	44.0
-47	47.0
-51	102.0
-53	53.0
-54	54.0
-57	57.0
-58	116.0
-64	64.0
-65	65.0
-66	66.0
-67	134.0
-69	69.0
-70	210.0
-72	144.0
-74	74.0
-76	152.0
-77	77.0
-78	78.0
-80	80.0
-82	82.0
-83	166.0
-84	168.0
-85	85.0
-86	86.0
-87	87.0
-90	270.0
-92	92.0
-95	190.0
-96	96.0
-97	194.0
-98	196.0
-100	200.0
-103	206.0
-104	208.0
-105	105.0
-111	111.0
-113	226.0
-114	114.0
-116	116.0
-118	236.0
-119	357.0
-120	240.0
-125	250.0
-126	126.0
-128	384.0
-129	258.0
-131	131.0
-133	133.0
-134	268.0
-136	136.0
-137	274.0
-138	552.0
-143	143.0
-145	145.0
-146	292.0
-149	298.0
-150	150.0
-152	304.0
-153	153.0
-155	155.0
-156	156.0
-157	157.0
-158	158.0
-160	160.0
-162	162.0
-163	163.0
-164	328.0
-165	330.0
-166	166.0
-167	501.0
-168	168.0
-169	676.0
-170	170.0
-172	344.0
-174	348.0
-175	350.0
-176	352.0
-177	177.0
-178	178.0
-179	358.0
-180	180.0
-181	181.0
-183	183.0
-186	186.0
-187	561.0
-189	189.0
-190	190.0
-191	382.0
-192	192.0
-193	579.0
-194	194.0
-195	390.0
-196	196.0
-197	394.0
-199	597.0
-200	400.0
-201	201.0
-202	202.0
-203	406.0
-205	410.0
-207	414.0
-208	624.0
-209	418.0
-213	426.0
-214	214.0
-216	432.0
-217	434.0
-218	218.0
-219	438.0
-221	442.0
-222	222.0
-223	446.0
-224	448.0
-226	226.0
-228	228.0
-229	458.0
-230	1150.0
-233	466.0
-235	235.0
-237	474.0
-238	476.0
-239	478.0
-241	241.0
-242	484.0
-244	244.0
-247	247.0
-248	248.0
-249	249.0
-252	252.0
-255	510.0
-256	512.0
-257	257.0
-258	258.0
-260	260.0
-262	262.0
-263	263.0
-265	530.0
-266	266.0
-272	544.0
-273	819.0
-274	274.0
-275	275.0
-277	1108.0
-278	556.0
-280	560.0
-281	562.0
-282	564.0
-283	283.0
-284	284.0
-285	285.0
-286	286.0
-287	287.0
-288	576.0
-289	289.0
-291	291.0
-292	292.0
-296	296.0
-298	894.0
-302	302.0
-305	305.0
-306	306.0
-307	614.0
-308	308.0
-309	618.0
-310	310.0
-311	933.0
-315	315.0
-316	948.0
-317	634.0
-318	954.0
-321	642.0
-322	644.0
-323	323.0
-325	650.0
-327	981.0
-331	662.0
-332	332.0
-333	666.0
-335	335.0
-336	336.0
-338	338.0
-339	339.0
-341	341.0
-342	684.0
-344	688.0
-345	345.0
-348	1740.0
-351	351.0
-353	706.0
-356	356.0
-360	360.0
-362	362.0
-364	364.0
-365	365.0
-366	366.0
-367	734.0
-368	368.0
-369	1107.0
-373	373.0
-374	374.0
-375	375.0
-377	377.0
-378	378.0
-379	379.0
-382	764.0
-384	1152.0
-386	386.0
-389	389.0
-392	392.0
-393	393.0
-394	394.0
-395	790.0
-396	1188.0
-397	794.0
-399	798.0
-400	400.0
-401	2005.0
-402	402.0
-403	1209.0
-404	808.0
-406	1624.0
-407	407.0
-409	1227.0
-411	411.0
-413	826.0
-414	828.0
-417	1251.0
-418	418.0
-419	419.0
-421	421.0
-424	848.0
-427	427.0
-429	858.0
-430	1290.0
-431	1293.0
-432	432.0
-435	435.0
-436	436.0
-437	437.0
-438	1314.0
-439	878.0
-443	443.0
-444	444.0
-446	446.0
-448	448.0
-449	449.0
-452	452.0
-453	453.0
-454	1362.0
-455	455.0
-457	457.0
-458	916.0
-459	918.0
-460	460.0
-462	924.0
-463	926.0
-466	1398.0
-467	467.0
-468	1872.0
-469	2345.0
-470	470.0
-472	472.0
-475	475.0
-477	477.0
-478	956.0
-479	479.0
-480	1440.0
-481	481.0
-482	482.0
-483	483.0
-484	484.0
-485	485.0
-487	487.0
-489	1956.0
-490	490.0
-491	491.0
-492	984.0
-493	493.0
-494	494.0
-495	495.0
-496	496.0
-497	497.0
-498	1494.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-c1a8cf4ef8060a4703b0affe40496169
new file mode 100644
index 0000000000000000000000000000000000000000..4953d093489ee79f5dd9f0cbcd91375cc0db53e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-c1a8cf4ef8060a4703b0affe40496169
@@ -0,0 +1,309 @@
+0	0.0
+2	2.0
+4	4.0
+5	15.0
+8	8.0
+9	9.0
+10	10.0
+11	11.0
+12	24.0
+15	30.0
+17	17.0
+18	36.0
+19	19.0
+20	20.0
+24	48.0
+26	52.0
+27	27.0
+28	28.0
+30	30.0
+33	33.0
+34	34.0
+35	105.0
+37	74.0
+41	41.0
+42	84.0
+43	43.0
+44	44.0
+47	47.0
+51	102.0
+53	53.0
+54	54.0
+57	57.0
+58	116.0
+64	64.0
+65	65.0
+66	66.0
+67	134.0
+69	69.0
+70	210.0
+72	144.0
+74	74.0
+76	152.0
+77	77.0
+78	78.0
+80	80.0
+82	82.0
+83	166.0
+84	168.0
+85	85.0
+86	86.0
+87	87.0
+90	270.0
+92	92.0
+95	190.0
+96	96.0
+97	194.0
+98	196.0
+100	200.0
+103	206.0
+104	208.0
+105	105.0
+111	111.0
+113	226.0
+114	114.0
+116	116.0
+118	236.0
+119	357.0
+120	240.0
+125	250.0
+126	126.0
+128	384.0
+129	258.0
+131	131.0
+133	133.0
+134	268.0
+136	136.0
+137	274.0
+138	552.0
+143	143.0
+145	145.0
+146	292.0
+149	298.0
+150	150.0
+152	304.0
+153	153.0
+155	155.0
+156	156.0
+157	157.0
+158	158.0
+160	160.0
+162	162.0
+163	163.0
+164	328.0
+165	330.0
+166	166.0
+167	501.0
+168	168.0
+169	676.0
+170	170.0
+172	344.0
+174	348.0
+175	350.0
+176	352.0
+177	177.0
+178	178.0
+179	358.0
+180	180.0
+181	181.0
+183	183.0
+186	186.0
+187	561.0
+189	189.0
+190	190.0
+191	382.0
+192	192.0
+193	579.0
+194	194.0
+195	390.0
+196	196.0
+197	394.0
+199	597.0
+200	400.0
+201	201.0
+202	202.0
+203	406.0
+205	410.0
+207	414.0
+208	624.0
+209	418.0
+213	426.0
+214	214.0
+216	432.0
+217	434.0
+218	218.0
+219	438.0
+221	442.0
+222	222.0
+223	446.0
+224	448.0
+226	226.0
+228	228.0
+229	458.0
+230	1150.0
+233	466.0
+235	235.0
+237	474.0
+238	476.0
+239	478.0
+241	241.0
+242	484.0
+244	244.0
+247	247.0
+248	248.0
+249	249.0
+252	252.0
+255	510.0
+256	512.0
+257	257.0
+258	258.0
+260	260.0
+262	262.0
+263	263.0
+265	530.0
+266	266.0
+272	544.0
+273	819.0
+274	274.0
+275	275.0
+277	1108.0
+278	556.0
+280	560.0
+281	562.0
+282	564.0
+283	283.0
+284	284.0
+285	285.0
+286	286.0
+287	287.0
+288	576.0
+289	289.0
+291	291.0
+292	292.0
+296	296.0
+298	894.0
+302	302.0
+305	305.0
+306	306.0
+307	614.0
+308	308.0
+309	618.0
+310	310.0
+311	933.0
+315	315.0
+316	948.0
+317	634.0
+318	954.0
+321	642.0
+322	644.0
+323	323.0
+325	650.0
+327	981.0
+331	662.0
+332	332.0
+333	666.0
+335	335.0
+336	336.0
+338	338.0
+339	339.0
+341	341.0
+342	684.0
+344	688.0
+345	345.0
+348	1740.0
+351	351.0
+353	706.0
+356	356.0
+360	360.0
+362	362.0
+364	364.0
+365	365.0
+366	366.0
+367	734.0
+368	368.0
+369	1107.0
+373	373.0
+374	374.0
+375	375.0
+377	377.0
+378	378.0
+379	379.0
+382	764.0
+384	1152.0
+386	386.0
+389	389.0
+392	392.0
+393	393.0
+394	394.0
+395	790.0
+396	1188.0
+397	794.0
+399	798.0
+400	400.0
+401	2005.0
+402	402.0
+403	1209.0
+404	808.0
+406	1624.0
+407	407.0
+409	1227.0
+411	411.0
+413	826.0
+414	828.0
+417	1251.0
+418	418.0
+419	419.0
+421	421.0
+424	848.0
+427	427.0
+429	858.0
+430	1290.0
+431	1293.0
+432	432.0
+435	435.0
+436	436.0
+437	437.0
+438	1314.0
+439	878.0
+443	443.0
+444	444.0
+446	446.0
+448	448.0
+449	449.0
+452	452.0
+453	453.0
+454	1362.0
+455	455.0
+457	457.0
+458	916.0
+459	918.0
+460	460.0
+462	924.0
+463	926.0
+466	1398.0
+467	467.0
+468	1872.0
+469	2345.0
+470	470.0
+472	472.0
+475	475.0
+477	477.0
+478	956.0
+479	479.0
+480	1440.0
+481	481.0
+482	482.0
+483	483.0
+484	484.0
+485	485.0
+487	487.0
+489	1956.0
+490	490.0
+491	491.0
+492	984.0
+493	493.0
+494	494.0
+495	495.0
+496	496.0
+497	497.0
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9
deleted file mode 100644
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9
+++ /dev/null
@@ -1,309 +0,0 @@
-0	0.0
-2	2.0
-4	4.0
-5	15.0
-8	8.0
-9	9.0
-10	10.0
-11	11.0
-12	24.0
-15	30.0
-17	17.0
-18	36.0
-19	19.0
-20	20.0
-24	48.0
-26	52.0
-27	27.0
-28	28.0
-30	30.0
-33	33.0
-34	34.0
-35	105.0
-37	74.0
-41	41.0
-42	84.0
-43	43.0
-44	44.0
-47	47.0
-51	102.0
-53	53.0
-54	54.0
-57	57.0
-58	116.0
-64	64.0
-65	65.0
-66	66.0
-67	134.0
-69	69.0
-70	210.0
-72	144.0
-74	74.0
-76	152.0
-77	77.0
-78	78.0
-80	80.0
-82	82.0
-83	166.0
-84	168.0
-85	85.0
-86	86.0
-87	87.0
-90	270.0
-92	92.0
-95	190.0
-96	96.0
-97	194.0
-98	196.0
-100	200.0
-103	206.0
-104	208.0
-105	105.0
-111	111.0
-113	226.0
-114	114.0
-116	116.0
-118	236.0
-119	357.0
-120	240.0
-125	250.0
-126	126.0
-128	384.0
-129	258.0
-131	131.0
-133	133.0
-134	268.0
-136	136.0
-137	274.0
-138	552.0
-143	143.0
-145	145.0
-146	292.0
-149	298.0
-150	150.0
-152	304.0
-153	153.0
-155	155.0
-156	156.0
-157	157.0
-158	158.0
-160	160.0
-162	162.0
-163	163.0
-164	328.0
-165	330.0
-166	166.0
-167	501.0
-168	168.0
-169	676.0
-170	170.0
-172	344.0
-174	348.0
-175	350.0
-176	352.0
-177	177.0
-178	178.0
-179	358.0
-180	180.0
-181	181.0
-183	183.0
-186	186.0
-187	561.0
-189	189.0
-190	190.0
-191	382.0
-192	192.0
-193	579.0
-194	194.0
-195	390.0
-196	196.0
-197	394.0
-199	597.0
-200	400.0
-201	201.0
-202	202.0
-203	406.0
-205	410.0
-207	414.0
-208	624.0
-209	418.0
-213	426.0
-214	214.0
-216	432.0
-217	434.0
-218	218.0
-219	438.0
-221	442.0
-222	222.0
-223	446.0
-224	448.0
-226	226.0
-228	228.0
-229	458.0
-230	1150.0
-233	466.0
-235	235.0
-237	474.0
-238	476.0
-239	478.0
-241	241.0
-242	484.0
-244	244.0
-247	247.0
-248	248.0
-249	249.0
-252	252.0
-255	510.0
-256	512.0
-257	257.0
-258	258.0
-260	260.0
-262	262.0
-263	263.0
-265	530.0
-266	266.0
-272	544.0
-273	819.0
-274	274.0
-275	275.0
-277	1108.0
-278	556.0
-280	560.0
-281	562.0
-282	564.0
-283	283.0
-284	284.0
-285	285.0
-286	286.0
-287	287.0
-288	576.0
-289	289.0
-291	291.0
-292	292.0
-296	296.0
-298	894.0
-302	302.0
-305	305.0
-306	306.0
-307	614.0
-308	308.0
-309	618.0
-310	310.0
-311	933.0
-315	315.0
-316	948.0
-317	634.0
-318	954.0
-321	642.0
-322	644.0
-323	323.0
-325	650.0
-327	981.0
-331	662.0
-332	332.0
-333	666.0
-335	335.0
-336	336.0
-338	338.0
-339	339.0
-341	341.0
-342	684.0
-344	688.0
-345	345.0
-348	1740.0
-351	351.0
-353	706.0
-356	356.0
-360	360.0
-362	362.0
-364	364.0
-365	365.0
-366	366.0
-367	734.0
-368	368.0
-369	1107.0
-373	373.0
-374	374.0
-375	375.0
-377	377.0
-378	378.0
-379	379.0
-382	764.0
-384	1152.0
-386	386.0
-389	389.0
-392	392.0
-393	393.0
-394	394.0
-395	790.0
-396	1188.0
-397	794.0
-399	798.0
-400	400.0
-401	2005.0
-402	402.0
-403	1209.0
-404	808.0
-406	1624.0
-407	407.0
-409	1227.0
-411	411.0
-413	826.0
-414	828.0
-417	1251.0
-418	418.0
-419	419.0
-421	421.0
-424	848.0
-427	427.0
-429	858.0
-430	1290.0
-431	1293.0
-432	432.0
-435	435.0
-436	436.0
-437	437.0
-438	1314.0
-439	878.0
-443	443.0
-444	444.0
-446	446.0
-448	448.0
-449	449.0
-452	452.0
-453	453.0
-454	1362.0
-455	455.0
-457	457.0
-458	916.0
-459	918.0
-460	460.0
-462	924.0
-463	926.0
-466	1398.0
-467	467.0
-468	1872.0
-469	2345.0
-470	470.0
-472	472.0
-475	475.0
-477	477.0
-478	956.0
-479	479.0
-480	1440.0
-481	481.0
-482	482.0
-483	483.0
-484	484.0
-485	485.0
-487	487.0
-489	1956.0
-490	490.0
-491	491.0
-492	984.0
-493	493.0
-494	494.0
-495	495.0
-496	496.0
-497	497.0
-498	1494.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-10-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby7_map_skew-10-6c26c5d39c4bdb61728defa9b44bfb52
new file mode 100644
index 0000000000000000000000000000000000000000..4953d093489ee79f5dd9f0cbcd91375cc0db53e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-10-6c26c5d39c4bdb61728defa9b44bfb52
@@ -0,0 +1,309 @@
+0	0.0
+2	2.0
+4	4.0
+5	15.0
+8	8.0
+9	9.0
+10	10.0
+11	11.0
+12	24.0
+15	30.0
+17	17.0
+18	36.0
+19	19.0
+20	20.0
+24	48.0
+26	52.0
+27	27.0
+28	28.0
+30	30.0
+33	33.0
+34	34.0
+35	105.0
+37	74.0
+41	41.0
+42	84.0
+43	43.0
+44	44.0
+47	47.0
+51	102.0
+53	53.0
+54	54.0
+57	57.0
+58	116.0
+64	64.0
+65	65.0
+66	66.0
+67	134.0
+69	69.0
+70	210.0
+72	144.0
+74	74.0
+76	152.0
+77	77.0
+78	78.0
+80	80.0
+82	82.0
+83	166.0
+84	168.0
+85	85.0
+86	86.0
+87	87.0
+90	270.0
+92	92.0
+95	190.0
+96	96.0
+97	194.0
+98	196.0
+100	200.0
+103	206.0
+104	208.0
+105	105.0
+111	111.0
+113	226.0
+114	114.0
+116	116.0
+118	236.0
+119	357.0
+120	240.0
+125	250.0
+126	126.0
+128	384.0
+129	258.0
+131	131.0
+133	133.0
+134	268.0
+136	136.0
+137	274.0
+138	552.0
+143	143.0
+145	145.0
+146	292.0
+149	298.0
+150	150.0
+152	304.0
+153	153.0
+155	155.0
+156	156.0
+157	157.0
+158	158.0
+160	160.0
+162	162.0
+163	163.0
+164	328.0
+165	330.0
+166	166.0
+167	501.0
+168	168.0
+169	676.0
+170	170.0
+172	344.0
+174	348.0
+175	350.0
+176	352.0
+177	177.0
+178	178.0
+179	358.0
+180	180.0
+181	181.0
+183	183.0
+186	186.0
+187	561.0
+189	189.0
+190	190.0
+191	382.0
+192	192.0
+193	579.0
+194	194.0
+195	390.0
+196	196.0
+197	394.0
+199	597.0
+200	400.0
+201	201.0
+202	202.0
+203	406.0
+205	410.0
+207	414.0
+208	624.0
+209	418.0
+213	426.0
+214	214.0
+216	432.0
+217	434.0
+218	218.0
+219	438.0
+221	442.0
+222	222.0
+223	446.0
+224	448.0
+226	226.0
+228	228.0
+229	458.0
+230	1150.0
+233	466.0
+235	235.0
+237	474.0
+238	476.0
+239	478.0
+241	241.0
+242	484.0
+244	244.0
+247	247.0
+248	248.0
+249	249.0
+252	252.0
+255	510.0
+256	512.0
+257	257.0
+258	258.0
+260	260.0
+262	262.0
+263	263.0
+265	530.0
+266	266.0
+272	544.0
+273	819.0
+274	274.0
+275	275.0
+277	1108.0
+278	556.0
+280	560.0
+281	562.0
+282	564.0
+283	283.0
+284	284.0
+285	285.0
+286	286.0
+287	287.0
+288	576.0
+289	289.0
+291	291.0
+292	292.0
+296	296.0
+298	894.0
+302	302.0
+305	305.0
+306	306.0
+307	614.0
+308	308.0
+309	618.0
+310	310.0
+311	933.0
+315	315.0
+316	948.0
+317	634.0
+318	954.0
+321	642.0
+322	644.0
+323	323.0
+325	650.0
+327	981.0
+331	662.0
+332	332.0
+333	666.0
+335	335.0
+336	336.0
+338	338.0
+339	339.0
+341	341.0
+342	684.0
+344	688.0
+345	345.0
+348	1740.0
+351	351.0
+353	706.0
+356	356.0
+360	360.0
+362	362.0
+364	364.0
+365	365.0
+366	366.0
+367	734.0
+368	368.0
+369	1107.0
+373	373.0
+374	374.0
+375	375.0
+377	377.0
+378	378.0
+379	379.0
+382	764.0
+384	1152.0
+386	386.0
+389	389.0
+392	392.0
+393	393.0
+394	394.0
+395	790.0
+396	1188.0
+397	794.0
+399	798.0
+400	400.0
+401	2005.0
+402	402.0
+403	1209.0
+404	808.0
+406	1624.0
+407	407.0
+409	1227.0
+411	411.0
+413	826.0
+414	828.0
+417	1251.0
+418	418.0
+419	419.0
+421	421.0
+424	848.0
+427	427.0
+429	858.0
+430	1290.0
+431	1293.0
+432	432.0
+435	435.0
+436	436.0
+437	437.0
+438	1314.0
+439	878.0
+443	443.0
+444	444.0
+446	446.0
+448	448.0
+449	449.0
+452	452.0
+453	453.0
+454	1362.0
+455	455.0
+457	457.0
+458	916.0
+459	918.0
+460	460.0
+462	924.0
+463	926.0
+466	1398.0
+467	467.0
+468	1872.0
+469	2345.0
+470	470.0
+472	472.0
+475	475.0
+477	477.0
+478	956.0
+479	479.0
+480	1440.0
+481	481.0
+482	482.0
+483	483.0
+484	484.0
+485	485.0
+487	487.0
+489	1956.0
+490	490.0
+491	491.0
+492	984.0
+493	493.0
+494	494.0
+495	495.0
+496	496.0
+497	497.0
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c
+++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403
+++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5
deleted file mode 100644
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ /dev/null
@@ -1,309 +0,0 @@
-0	0.0
-2	2.0
-4	4.0
-5	15.0
-8	8.0
-9	9.0
-10	10.0
-11	11.0
-12	24.0
-15	30.0
-17	17.0
-18	36.0
-19	19.0
-20	20.0
-24	48.0
-26	52.0
-27	27.0
-28	28.0
-30	30.0
-33	33.0
-34	34.0
-35	105.0
-37	74.0
-41	41.0
-42	84.0
-43	43.0
-44	44.0
-47	47.0
-51	102.0
-53	53.0
-54	54.0
-57	57.0
-58	116.0
-64	64.0
-65	65.0
-66	66.0
-67	134.0
-69	69.0
-70	210.0
-72	144.0
-74	74.0
-76	152.0
-77	77.0
-78	78.0
-80	80.0
-82	82.0
-83	166.0
-84	168.0
-85	85.0
-86	86.0
-87	87.0
-90	270.0
-92	92.0
-95	190.0
-96	96.0
-97	194.0
-98	196.0
-100	200.0
-103	206.0
-104	208.0
-105	105.0
-111	111.0
-113	226.0
-114	114.0
-116	116.0
-118	236.0
-119	357.0
-120	240.0
-125	250.0
-126	126.0
-128	384.0
-129	258.0
-131	131.0
-133	133.0
-134	268.0
-136	136.0
-137	274.0
-138	552.0
-143	143.0
-145	145.0
-146	292.0
-149	298.0
-150	150.0
-152	304.0
-153	153.0
-155	155.0
-156	156.0
-157	157.0
-158	158.0
-160	160.0
-162	162.0
-163	163.0
-164	328.0
-165	330.0
-166	166.0
-167	501.0
-168	168.0
-169	676.0
-170	170.0
-172	344.0
-174	348.0
-175	350.0
-176	352.0
-177	177.0
-178	178.0
-179	358.0
-180	180.0
-181	181.0
-183	183.0
-186	186.0
-187	561.0
-189	189.0
-190	190.0
-191	382.0
-192	192.0
-193	579.0
-194	194.0
-195	390.0
-196	196.0
-197	394.0
-199	597.0
-200	400.0
-201	201.0
-202	202.0
-203	406.0
-205	410.0
-207	414.0
-208	624.0
-209	418.0
-213	426.0
-214	214.0
-216	432.0
-217	434.0
-218	218.0
-219	438.0
-221	442.0
-222	222.0
-223	446.0
-224	448.0
-226	226.0
-228	228.0
-229	458.0
-230	1150.0
-233	466.0
-235	235.0
-237	474.0
-238	476.0
-239	478.0
-241	241.0
-242	484.0
-244	244.0
-247	247.0
-248	248.0
-249	249.0
-252	252.0
-255	510.0
-256	512.0
-257	257.0
-258	258.0
-260	260.0
-262	262.0
-263	263.0
-265	530.0
-266	266.0
-272	544.0
-273	819.0
-274	274.0
-275	275.0
-277	1108.0
-278	556.0
-280	560.0
-281	562.0
-282	564.0
-283	283.0
-284	284.0
-285	285.0
-286	286.0
-287	287.0
-288	576.0
-289	289.0
-291	291.0
-292	292.0
-296	296.0
-298	894.0
-302	302.0
-305	305.0
-306	306.0
-307	614.0
-308	308.0
-309	618.0
-310	310.0
-311	933.0
-315	315.0
-316	948.0
-317	634.0
-318	954.0
-321	642.0
-322	644.0
-323	323.0
-325	650.0
-327	981.0
-331	662.0
-332	332.0
-333	666.0
-335	335.0
-336	336.0
-338	338.0
-339	339.0
-341	341.0
-342	684.0
-344	688.0
-345	345.0
-348	1740.0
-351	351.0
-353	706.0
-356	356.0
-360	360.0
-362	362.0
-364	364.0
-365	365.0
-366	366.0
-367	734.0
-368	368.0
-369	1107.0
-373	373.0
-374	374.0
-375	375.0
-377	377.0
-378	378.0
-379	379.0
-382	764.0
-384	1152.0
-386	386.0
-389	389.0
-392	392.0
-393	393.0
-394	394.0
-395	790.0
-396	1188.0
-397	794.0
-399	798.0
-400	400.0
-401	2005.0
-402	402.0
-403	1209.0
-404	808.0
-406	1624.0
-407	407.0
-409	1227.0
-411	411.0
-413	826.0
-414	828.0
-417	1251.0
-418	418.0
-419	419.0
-421	421.0
-424	848.0
-427	427.0
-429	858.0
-430	1290.0
-431	1293.0
-432	432.0
-435	435.0
-436	436.0
-437	437.0
-438	1314.0
-439	878.0
-443	443.0
-444	444.0
-446	446.0
-448	448.0
-449	449.0
-452	452.0
-453	453.0
-454	1362.0
-455	455.0
-457	457.0
-458	916.0
-459	918.0
-460	460.0
-462	924.0
-463	926.0
-466	1398.0
-467	467.0
-468	1872.0
-469	2345.0
-470	470.0
-472	472.0
-475	475.0
-477	477.0
-478	956.0
-479	479.0
-480	1440.0
-481	481.0
-482	482.0
-483	483.0
-484	484.0
-485	485.0
-487	487.0
-489	1956.0
-490	490.0
-491	491.0
-492	984.0
-493	493.0
-494	494.0
-495	495.0
-496	496.0
-497	497.0
-498	1494.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-9-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby7_map_skew-9-c1a8cf4ef8060a4703b0affe40496169
new file mode 100644
index 0000000000000000000000000000000000000000..4953d093489ee79f5dd9f0cbcd91375cc0db53e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-9-c1a8cf4ef8060a4703b0affe40496169
@@ -0,0 +1,309 @@
+0	0.0
+2	2.0
+4	4.0
+5	15.0
+8	8.0
+9	9.0
+10	10.0
+11	11.0
+12	24.0
+15	30.0
+17	17.0
+18	36.0
+19	19.0
+20	20.0
+24	48.0
+26	52.0
+27	27.0
+28	28.0
+30	30.0
+33	33.0
+34	34.0
+35	105.0
+37	74.0
+41	41.0
+42	84.0
+43	43.0
+44	44.0
+47	47.0
+51	102.0
+53	53.0
+54	54.0
+57	57.0
+58	116.0
+64	64.0
+65	65.0
+66	66.0
+67	134.0
+69	69.0
+70	210.0
+72	144.0
+74	74.0
+76	152.0
+77	77.0
+78	78.0
+80	80.0
+82	82.0
+83	166.0
+84	168.0
+85	85.0
+86	86.0
+87	87.0
+90	270.0
+92	92.0
+95	190.0
+96	96.0
+97	194.0
+98	196.0
+100	200.0
+103	206.0
+104	208.0
+105	105.0
+111	111.0
+113	226.0
+114	114.0
+116	116.0
+118	236.0
+119	357.0
+120	240.0
+125	250.0
+126	126.0
+128	384.0
+129	258.0
+131	131.0
+133	133.0
+134	268.0
+136	136.0
+137	274.0
+138	552.0
+143	143.0
+145	145.0
+146	292.0
+149	298.0
+150	150.0
+152	304.0
+153	153.0
+155	155.0
+156	156.0
+157	157.0
+158	158.0
+160	160.0
+162	162.0
+163	163.0
+164	328.0
+165	330.0
+166	166.0
+167	501.0
+168	168.0
+169	676.0
+170	170.0
+172	344.0
+174	348.0
+175	350.0
+176	352.0
+177	177.0
+178	178.0
+179	358.0
+180	180.0
+181	181.0
+183	183.0
+186	186.0
+187	561.0
+189	189.0
+190	190.0
+191	382.0
+192	192.0
+193	579.0
+194	194.0
+195	390.0
+196	196.0
+197	394.0
+199	597.0
+200	400.0
+201	201.0
+202	202.0
+203	406.0
+205	410.0
+207	414.0
+208	624.0
+209	418.0
+213	426.0
+214	214.0
+216	432.0
+217	434.0
+218	218.0
+219	438.0
+221	442.0
+222	222.0
+223	446.0
+224	448.0
+226	226.0
+228	228.0
+229	458.0
+230	1150.0
+233	466.0
+235	235.0
+237	474.0
+238	476.0
+239	478.0
+241	241.0
+242	484.0
+244	244.0
+247	247.0
+248	248.0
+249	249.0
+252	252.0
+255	510.0
+256	512.0
+257	257.0
+258	258.0
+260	260.0
+262	262.0
+263	263.0
+265	530.0
+266	266.0
+272	544.0
+273	819.0
+274	274.0
+275	275.0
+277	1108.0
+278	556.0
+280	560.0
+281	562.0
+282	564.0
+283	283.0
+284	284.0
+285	285.0
+286	286.0
+287	287.0
+288	576.0
+289	289.0
+291	291.0
+292	292.0
+296	296.0
+298	894.0
+302	302.0
+305	305.0
+306	306.0
+307	614.0
+308	308.0
+309	618.0
+310	310.0
+311	933.0
+315	315.0
+316	948.0
+317	634.0
+318	954.0
+321	642.0
+322	644.0
+323	323.0
+325	650.0
+327	981.0
+331	662.0
+332	332.0
+333	666.0
+335	335.0
+336	336.0
+338	338.0
+339	339.0
+341	341.0
+342	684.0
+344	688.0
+345	345.0
+348	1740.0
+351	351.0
+353	706.0
+356	356.0
+360	360.0
+362	362.0
+364	364.0
+365	365.0
+366	366.0
+367	734.0
+368	368.0
+369	1107.0
+373	373.0
+374	374.0
+375	375.0
+377	377.0
+378	378.0
+379	379.0
+382	764.0
+384	1152.0
+386	386.0
+389	389.0
+392	392.0
+393	393.0
+394	394.0
+395	790.0
+396	1188.0
+397	794.0
+399	798.0
+400	400.0
+401	2005.0
+402	402.0
+403	1209.0
+404	808.0
+406	1624.0
+407	407.0
+409	1227.0
+411	411.0
+413	826.0
+414	828.0
+417	1251.0
+418	418.0
+419	419.0
+421	421.0
+424	848.0
+427	427.0
+429	858.0
+430	1290.0
+431	1293.0
+432	432.0
+435	435.0
+436	436.0
+437	437.0
+438	1314.0
+439	878.0
+443	443.0
+444	444.0
+446	446.0
+448	448.0
+449	449.0
+452	452.0
+453	453.0
+454	1362.0
+455	455.0
+457	457.0
+458	916.0
+459	918.0
+460	460.0
+462	924.0
+463	926.0
+466	1398.0
+467	467.0
+468	1872.0
+469	2345.0
+470	470.0
+472	472.0
+475	475.0
+477	477.0
+478	956.0
+479	479.0
+480	1440.0
+481	481.0
+482	482.0
+483	483.0
+484	484.0
+485	485.0
+487	487.0
+489	1956.0
+490	490.0
+491	491.0
+492	984.0
+493	493.0
+494	494.0
+495	495.0
+496	496.0
+497	497.0
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5
deleted file mode 100644
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ /dev/null
@@ -1,309 +0,0 @@
-0	0.0
-2	2.0
-4	4.0
-5	15.0
-8	8.0
-9	9.0
-10	10.0
-11	11.0
-12	24.0
-15	30.0
-17	17.0
-18	36.0
-19	19.0
-20	20.0
-24	48.0
-26	52.0
-27	27.0
-28	28.0
-30	30.0
-33	33.0
-34	34.0
-35	105.0
-37	74.0
-41	41.0
-42	84.0
-43	43.0
-44	44.0
-47	47.0
-51	102.0
-53	53.0
-54	54.0
-57	57.0
-58	116.0
-64	64.0
-65	65.0
-66	66.0
-67	134.0
-69	69.0
-70	210.0
-72	144.0
-74	74.0
-76	152.0
-77	77.0
-78	78.0
-80	80.0
-82	82.0
-83	166.0
-84	168.0
-85	85.0
-86	86.0
-87	87.0
-90	270.0
-92	92.0
-95	190.0
-96	96.0
-97	194.0
-98	196.0
-100	200.0
-103	206.0
-104	208.0
-105	105.0
-111	111.0
-113	226.0
-114	114.0
-116	116.0
-118	236.0
-119	357.0
-120	240.0
-125	250.0
-126	126.0
-128	384.0
-129	258.0
-131	131.0
-133	133.0
-134	268.0
-136	136.0
-137	274.0
-138	552.0
-143	143.0
-145	145.0
-146	292.0
-149	298.0
-150	150.0
-152	304.0
-153	153.0
-155	155.0
-156	156.0
-157	157.0
-158	158.0
-160	160.0
-162	162.0
-163	163.0
-164	328.0
-165	330.0
-166	166.0
-167	501.0
-168	168.0
-169	676.0
-170	170.0
-172	344.0
-174	348.0
-175	350.0
-176	352.0
-177	177.0
-178	178.0
-179	358.0
-180	180.0
-181	181.0
-183	183.0
-186	186.0
-187	561.0
-189	189.0
-190	190.0
-191	382.0
-192	192.0
-193	579.0
-194	194.0
-195	390.0
-196	196.0
-197	394.0
-199	597.0
-200	400.0
-201	201.0
-202	202.0
-203	406.0
-205	410.0
-207	414.0
-208	624.0
-209	418.0
-213	426.0
-214	214.0
-216	432.0
-217	434.0
-218	218.0
-219	438.0
-221	442.0
-222	222.0
-223	446.0
-224	448.0
-226	226.0
-228	228.0
-229	458.0
-230	1150.0
-233	466.0
-235	235.0
-237	474.0
-238	476.0
-239	478.0
-241	241.0
-242	484.0
-244	244.0
-247	247.0
-248	248.0
-249	249.0
-252	252.0
-255	510.0
-256	512.0
-257	257.0
-258	258.0
-260	260.0
-262	262.0
-263	263.0
-265	530.0
-266	266.0
-272	544.0
-273	819.0
-274	274.0
-275	275.0
-277	1108.0
-278	556.0
-280	560.0
-281	562.0
-282	564.0
-283	283.0
-284	284.0
-285	285.0
-286	286.0
-287	287.0
-288	576.0
-289	289.0
-291	291.0
-292	292.0
-296	296.0
-298	894.0
-302	302.0
-305	305.0
-306	306.0
-307	614.0
-308	308.0
-309	618.0
-310	310.0
-311	933.0
-315	315.0
-316	948.0
-317	634.0
-318	954.0
-321	642.0
-322	644.0
-323	323.0
-325	650.0
-327	981.0
-331	662.0
-332	332.0
-333	666.0
-335	335.0
-336	336.0
-338	338.0
-339	339.0
-341	341.0
-342	684.0
-344	688.0
-345	345.0
-348	1740.0
-351	351.0
-353	706.0
-356	356.0
-360	360.0
-362	362.0
-364	364.0
-365	365.0
-366	366.0
-367	734.0
-368	368.0
-369	1107.0
-373	373.0
-374	374.0
-375	375.0
-377	377.0
-378	378.0
-379	379.0
-382	764.0
-384	1152.0
-386	386.0
-389	389.0
-392	392.0
-393	393.0
-394	394.0
-395	790.0
-396	1188.0
-397	794.0
-399	798.0
-400	400.0
-401	2005.0
-402	402.0
-403	1209.0
-404	808.0
-406	1624.0
-407	407.0
-409	1227.0
-411	411.0
-413	826.0
-414	828.0
-417	1251.0
-418	418.0
-419	419.0
-421	421.0
-424	848.0
-427	427.0
-429	858.0
-430	1290.0
-431	1293.0
-432	432.0
-435	435.0
-436	436.0
-437	437.0
-438	1314.0
-439	878.0
-443	443.0
-444	444.0
-446	446.0
-448	448.0
-449	449.0
-452	452.0
-453	453.0
-454	1362.0
-455	455.0
-457	457.0
-458	916.0
-459	918.0
-460	460.0
-462	924.0
-463	926.0
-466	1398.0
-467	467.0
-468	1872.0
-469	2345.0
-470	470.0
-472	472.0
-475	475.0
-477	477.0
-478	956.0
-479	479.0
-480	1440.0
-481	481.0
-482	482.0
-483	483.0
-484	484.0
-485	485.0
-487	487.0
-489	1956.0
-490	490.0
-491	491.0
-492	984.0
-493	493.0
-494	494.0
-495	495.0
-496	496.0
-497	497.0
-498	1494.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-10-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby7_noskew-10-c1a8cf4ef8060a4703b0affe40496169
new file mode 100644
index 0000000000000000000000000000000000000000..4953d093489ee79f5dd9f0cbcd91375cc0db53e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew-10-c1a8cf4ef8060a4703b0affe40496169
@@ -0,0 +1,309 @@
+0	0.0
+2	2.0
+4	4.0
+5	15.0
+8	8.0
+9	9.0
+10	10.0
+11	11.0
+12	24.0
+15	30.0
+17	17.0
+18	36.0
+19	19.0
+20	20.0
+24	48.0
+26	52.0
+27	27.0
+28	28.0
+30	30.0
+33	33.0
+34	34.0
+35	105.0
+37	74.0
+41	41.0
+42	84.0
+43	43.0
+44	44.0
+47	47.0
+51	102.0
+53	53.0
+54	54.0
+57	57.0
+58	116.0
+64	64.0
+65	65.0
+66	66.0
+67	134.0
+69	69.0
+70	210.0
+72	144.0
+74	74.0
+76	152.0
+77	77.0
+78	78.0
+80	80.0
+82	82.0
+83	166.0
+84	168.0
+85	85.0
+86	86.0
+87	87.0
+90	270.0
+92	92.0
+95	190.0
+96	96.0
+97	194.0
+98	196.0
+100	200.0
+103	206.0
+104	208.0
+105	105.0
+111	111.0
+113	226.0
+114	114.0
+116	116.0
+118	236.0
+119	357.0
+120	240.0
+125	250.0
+126	126.0
+128	384.0
+129	258.0
+131	131.0
+133	133.0
+134	268.0
+136	136.0
+137	274.0
+138	552.0
+143	143.0
+145	145.0
+146	292.0
+149	298.0
+150	150.0
+152	304.0
+153	153.0
+155	155.0
+156	156.0
+157	157.0
+158	158.0
+160	160.0
+162	162.0
+163	163.0
+164	328.0
+165	330.0
+166	166.0
+167	501.0
+168	168.0
+169	676.0
+170	170.0
+172	344.0
+174	348.0
+175	350.0
+176	352.0
+177	177.0
+178	178.0
+179	358.0
+180	180.0
+181	181.0
+183	183.0
+186	186.0
+187	561.0
+189	189.0
+190	190.0
+191	382.0
+192	192.0
+193	579.0
+194	194.0
+195	390.0
+196	196.0
+197	394.0
+199	597.0
+200	400.0
+201	201.0
+202	202.0
+203	406.0
+205	410.0
+207	414.0
+208	624.0
+209	418.0
+213	426.0
+214	214.0
+216	432.0
+217	434.0
+218	218.0
+219	438.0
+221	442.0
+222	222.0
+223	446.0
+224	448.0
+226	226.0
+228	228.0
+229	458.0
+230	1150.0
+233	466.0
+235	235.0
+237	474.0
+238	476.0
+239	478.0
+241	241.0
+242	484.0
+244	244.0
+247	247.0
+248	248.0
+249	249.0
+252	252.0
+255	510.0
+256	512.0
+257	257.0
+258	258.0
+260	260.0
+262	262.0
+263	263.0
+265	530.0
+266	266.0
+272	544.0
+273	819.0
+274	274.0
+275	275.0
+277	1108.0
+278	556.0
+280	560.0
+281	562.0
+282	564.0
+283	283.0
+284	284.0
+285	285.0
+286	286.0
+287	287.0
+288	576.0
+289	289.0
+291	291.0
+292	292.0
+296	296.0
+298	894.0
+302	302.0
+305	305.0
+306	306.0
+307	614.0
+308	308.0
+309	618.0
+310	310.0
+311	933.0
+315	315.0
+316	948.0
+317	634.0
+318	954.0
+321	642.0
+322	644.0
+323	323.0
+325	650.0
+327	981.0
+331	662.0
+332	332.0
+333	666.0
+335	335.0
+336	336.0
+338	338.0
+339	339.0
+341	341.0
+342	684.0
+344	688.0
+345	345.0
+348	1740.0
+351	351.0
+353	706.0
+356	356.0
+360	360.0
+362	362.0
+364	364.0
+365	365.0
+366	366.0
+367	734.0
+368	368.0
+369	1107.0
+373	373.0
+374	374.0
+375	375.0
+377	377.0
+378	378.0
+379	379.0
+382	764.0
+384	1152.0
+386	386.0
+389	389.0
+392	392.0
+393	393.0
+394	394.0
+395	790.0
+396	1188.0
+397	794.0
+399	798.0
+400	400.0
+401	2005.0
+402	402.0
+403	1209.0
+404	808.0
+406	1624.0
+407	407.0
+409	1227.0
+411	411.0
+413	826.0
+414	828.0
+417	1251.0
+418	418.0
+419	419.0
+421	421.0
+424	848.0
+427	427.0
+429	858.0
+430	1290.0
+431	1293.0
+432	432.0
+435	435.0
+436	436.0
+437	437.0
+438	1314.0
+439	878.0
+443	443.0
+444	444.0
+446	446.0
+448	448.0
+449	449.0
+452	452.0
+453	453.0
+454	1362.0
+455	455.0
+457	457.0
+458	916.0
+459	918.0
+460	460.0
+462	924.0
+463	926.0
+466	1398.0
+467	467.0
+468	1872.0
+469	2345.0
+470	470.0
+472	472.0
+475	475.0
+477	477.0
+478	956.0
+479	479.0
+480	1440.0
+481	481.0
+482	482.0
+483	483.0
+484	484.0
+485	485.0
+487	487.0
+489	1956.0
+490	490.0
+491	491.0
+492	984.0
+493	493.0
+494	494.0
+495	495.0
+496	496.0
+497	497.0
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9
deleted file mode 100644
index 339756be98e73c6ca0cab0dff02f69b9813eb15a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9
+++ /dev/null
@@ -1,309 +0,0 @@
-0	0.0
-2	2.0
-4	4.0
-5	15.0
-8	8.0
-9	9.0
-10	10.0
-11	11.0
-12	24.0
-15	30.0
-17	17.0
-18	36.0
-19	19.0
-20	20.0
-24	48.0
-26	52.0
-27	27.0
-28	28.0
-30	30.0
-33	33.0
-34	34.0
-35	105.0
-37	74.0
-41	41.0
-42	84.0
-43	43.0
-44	44.0
-47	47.0
-51	102.0
-53	53.0
-54	54.0
-57	57.0
-58	116.0
-64	64.0
-65	65.0
-66	66.0
-67	134.0
-69	69.0
-70	210.0
-72	144.0
-74	74.0
-76	152.0
-77	77.0
-78	78.0
-80	80.0
-82	82.0
-83	166.0
-84	168.0
-85	85.0
-86	86.0
-87	87.0
-90	270.0
-92	92.0
-95	190.0
-96	96.0
-97	194.0
-98	196.0
-100	200.0
-103	206.0
-104	208.0
-105	105.0
-111	111.0
-113	226.0
-114	114.0
-116	116.0
-118	236.0
-119	357.0
-120	240.0
-125	250.0
-126	126.0
-128	384.0
-129	258.0
-131	131.0
-133	133.0
-134	268.0
-136	136.0
-137	274.0
-138	552.0
-143	143.0
-145	145.0
-146	292.0
-149	298.0
-150	150.0
-152	304.0
-153	153.0
-155	155.0
-156	156.0
-157	157.0
-158	158.0
-160	160.0
-162	162.0
-163	163.0
-164	328.0
-165	330.0
-166	166.0
-167	501.0
-168	168.0
-169	676.0
-170	170.0
-172	344.0
-174	348.0
-175	350.0
-176	352.0
-177	177.0
-178	178.0
-179	358.0
-180	180.0
-181	181.0
-183	183.0
-186	186.0
-187	561.0
-189	189.0
-190	190.0
-191	382.0
-192	192.0
-193	579.0
-194	194.0
-195	390.0
-196	196.0
-197	394.0
-199	597.0
-200	400.0
-201	201.0
-202	202.0
-203	406.0
-205	410.0
-207	414.0
-208	624.0
-209	418.0
-213	426.0
-214	214.0
-216	432.0
-217	434.0
-218	218.0
-219	438.0
-221	442.0
-222	222.0
-223	446.0
-224	448.0
-226	226.0
-228	228.0
-229	458.0
-230	1150.0
-233	466.0
-235	235.0
-237	474.0
-238	476.0
-239	478.0
-241	241.0
-242	484.0
-244	244.0
-247	247.0
-248	248.0
-249	249.0
-252	252.0
-255	510.0
-256	512.0
-257	257.0
-258	258.0
-260	260.0
-262	262.0
-263	263.0
-265	530.0
-266	266.0
-272	544.0
-273	819.0
-274	274.0
-275	275.0
-277	1108.0
-278	556.0
-280	560.0
-281	562.0
-282	564.0
-283	283.0
-284	284.0
-285	285.0
-286	286.0
-287	287.0
-288	576.0
-289	289.0
-291	291.0
-292	292.0
-296	296.0
-298	894.0
-302	302.0
-305	305.0
-306	306.0
-307	614.0
-308	308.0
-309	618.0
-310	310.0
-311	933.0
-315	315.0
-316	948.0
-317	634.0
-318	954.0
-321	642.0
-322	644.0
-323	323.0
-325	650.0
-327	981.0
-331	662.0
-332	332.0
-333	666.0
-335	335.0
-336	336.0
-338	338.0
-339	339.0
-341	341.0
-342	684.0
-344	688.0
-345	345.0
-348	1740.0
-351	351.0
-353	706.0
-356	356.0
-360	360.0
-362	362.0
-364	364.0
-365	365.0
-366	366.0
-367	734.0
-368	368.0
-369	1107.0
-373	373.0
-374	374.0
-375	375.0
-377	377.0
-378	378.0
-379	379.0
-382	764.0
-384	1152.0
-386	386.0
-389	389.0
-392	392.0
-393	393.0
-394	394.0
-395	790.0
-396	1188.0
-397	794.0
-399	798.0
-400	400.0
-401	2005.0
-402	402.0
-403	1209.0
-404	808.0
-406	1624.0
-407	407.0
-409	1227.0
-411	411.0
-413	826.0
-414	828.0
-417	1251.0
-418	418.0
-419	419.0
-421	421.0
-424	848.0
-427	427.0
-429	858.0
-430	1290.0
-431	1293.0
-432	432.0
-435	435.0
-436	436.0
-437	437.0
-438	1314.0
-439	878.0
-443	443.0
-444	444.0
-446	446.0
-448	448.0
-449	449.0
-452	452.0
-453	453.0
-454	1362.0
-455	455.0
-457	457.0
-458	916.0
-459	918.0
-460	460.0
-462	924.0
-463	926.0
-466	1398.0
-467	467.0
-468	1872.0
-469	2345.0
-470	470.0
-472	472.0
-475	475.0
-477	477.0
-478	956.0
-479	479.0
-480	1440.0
-481	481.0
-482	482.0
-483	483.0
-484	484.0
-485	485.0
-487	487.0
-489	1956.0
-490	490.0
-491	491.0
-492	984.0
-493	493.0
-494	494.0
-495	495.0
-496	496.0
-497	497.0
-498	1494.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-11-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby7_noskew-11-6c26c5d39c4bdb61728defa9b44bfb52
new file mode 100644
index 0000000000000000000000000000000000000000..4953d093489ee79f5dd9f0cbcd91375cc0db53e9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew-11-6c26c5d39c4bdb61728defa9b44bfb52
@@ -0,0 +1,309 @@
+0	0.0
+2	2.0
+4	4.0
+5	15.0
+8	8.0
+9	9.0
+10	10.0
+11	11.0
+12	24.0
+15	30.0
+17	17.0
+18	36.0
+19	19.0
+20	20.0
+24	48.0
+26	52.0
+27	27.0
+28	28.0
+30	30.0
+33	33.0
+34	34.0
+35	105.0
+37	74.0
+41	41.0
+42	84.0
+43	43.0
+44	44.0
+47	47.0
+51	102.0
+53	53.0
+54	54.0
+57	57.0
+58	116.0
+64	64.0
+65	65.0
+66	66.0
+67	134.0
+69	69.0
+70	210.0
+72	144.0
+74	74.0
+76	152.0
+77	77.0
+78	78.0
+80	80.0
+82	82.0
+83	166.0
+84	168.0
+85	85.0
+86	86.0
+87	87.0
+90	270.0
+92	92.0
+95	190.0
+96	96.0
+97	194.0
+98	196.0
+100	200.0
+103	206.0
+104	208.0
+105	105.0
+111	111.0
+113	226.0
+114	114.0
+116	116.0
+118	236.0
+119	357.0
+120	240.0
+125	250.0
+126	126.0
+128	384.0
+129	258.0
+131	131.0
+133	133.0
+134	268.0
+136	136.0
+137	274.0
+138	552.0
+143	143.0
+145	145.0
+146	292.0
+149	298.0
+150	150.0
+152	304.0
+153	153.0
+155	155.0
+156	156.0
+157	157.0
+158	158.0
+160	160.0
+162	162.0
+163	163.0
+164	328.0
+165	330.0
+166	166.0
+167	501.0
+168	168.0
+169	676.0
+170	170.0
+172	344.0
+174	348.0
+175	350.0
+176	352.0
+177	177.0
+178	178.0
+179	358.0
+180	180.0
+181	181.0
+183	183.0
+186	186.0
+187	561.0
+189	189.0
+190	190.0
+191	382.0
+192	192.0
+193	579.0
+194	194.0
+195	390.0
+196	196.0
+197	394.0
+199	597.0
+200	400.0
+201	201.0
+202	202.0
+203	406.0
+205	410.0
+207	414.0
+208	624.0
+209	418.0
+213	426.0
+214	214.0
+216	432.0
+217	434.0
+218	218.0
+219	438.0
+221	442.0
+222	222.0
+223	446.0
+224	448.0
+226	226.0
+228	228.0
+229	458.0
+230	1150.0
+233	466.0
+235	235.0
+237	474.0
+238	476.0
+239	478.0
+241	241.0
+242	484.0
+244	244.0
+247	247.0
+248	248.0
+249	249.0
+252	252.0
+255	510.0
+256	512.0
+257	257.0
+258	258.0
+260	260.0
+262	262.0
+263	263.0
+265	530.0
+266	266.0
+272	544.0
+273	819.0
+274	274.0
+275	275.0
+277	1108.0
+278	556.0
+280	560.0
+281	562.0
+282	564.0
+283	283.0
+284	284.0
+285	285.0
+286	286.0
+287	287.0
+288	576.0
+289	289.0
+291	291.0
+292	292.0
+296	296.0
+298	894.0
+302	302.0
+305	305.0
+306	306.0
+307	614.0
+308	308.0
+309	618.0
+310	310.0
+311	933.0
+315	315.0
+316	948.0
+317	634.0
+318	954.0
+321	642.0
+322	644.0
+323	323.0
+325	650.0
+327	981.0
+331	662.0
+332	332.0
+333	666.0
+335	335.0
+336	336.0
+338	338.0
+339	339.0
+341	341.0
+342	684.0
+344	688.0
+345	345.0
+348	1740.0
+351	351.0
+353	706.0
+356	356.0
+360	360.0
+362	362.0
+364	364.0
+365	365.0
+366	366.0
+367	734.0
+368	368.0
+369	1107.0
+373	373.0
+374	374.0
+375	375.0
+377	377.0
+378	378.0
+379	379.0
+382	764.0
+384	1152.0
+386	386.0
+389	389.0
+392	392.0
+393	393.0
+394	394.0
+395	790.0
+396	1188.0
+397	794.0
+399	798.0
+400	400.0
+401	2005.0
+402	402.0
+403	1209.0
+404	808.0
+406	1624.0
+407	407.0
+409	1227.0
+411	411.0
+413	826.0
+414	828.0
+417	1251.0
+418	418.0
+419	419.0
+421	421.0
+424	848.0
+427	427.0
+429	858.0
+430	1290.0
+431	1293.0
+432	432.0
+435	435.0
+436	436.0
+437	437.0
+438	1314.0
+439	878.0
+443	443.0
+444	444.0
+446	446.0
+448	448.0
+449	449.0
+452	452.0
+453	453.0
+454	1362.0
+455	455.0
+457	457.0
+458	916.0
+459	918.0
+460	460.0
+462	924.0
+463	926.0
+466	1398.0
+467	467.0
+468	1872.0
+469	2345.0
+470	470.0
+472	472.0
+475	475.0
+477	477.0
+478	956.0
+479	479.0
+480	1440.0
+481	481.0
+482	482.0
+483	483.0
+484	484.0
+485	485.0
+487	487.0
+489	1956.0
+490	490.0
+491	491.0
+492	984.0
+493	493.0
+494	494.0
+495	495.0
+496	496.0
+497	497.0
+498	1494.0
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5
index dfca4e169cbe8368ad3f24670b5d176b4cbee1cd..951e74db0fe23bf2c6fe1439f3d400574904526f 100644
--- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5
@@ -7,4 +7,4 @@
 10	10.0
 11	11.0
 12	24.0
-15	30.0
\ No newline at end of file
+15	30.0
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-ab2390a3b0ba76907a7bee390b2924a1
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110
rename to sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-ab2390a3b0ba76907a7bee390b2924a1
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73
index dfca4e169cbe8368ad3f24670b5d176b4cbee1cd..951e74db0fe23bf2c6fe1439f3d400574904526f 100644
--- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73
+++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73
@@ -7,4 +7,4 @@
 10	10.0
 11	11.0
 12	24.0
-15	30.0
\ No newline at end of file
+15	30.0
diff --git a/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5
index 326493a1d8cb65c016f9b00f4a47e80cba58c2d2..8bfbbf1e69f57b941cd53fc5ce6c786930f726be 100644
--- a/sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ b/sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5
@@ -306,4 +306,4 @@
 495	1
 496	1
 497	1
-498	1
\ No newline at end of file
+498	1
diff --git a/sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9
index 326493a1d8cb65c016f9b00f4a47e80cba58c2d2..8bfbbf1e69f57b941cd53fc5ce6c786930f726be 100644
--- a/sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9
+++ b/sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9
@@ -306,4 +306,4 @@
 495	1
 496	1
 497	1
-498	1
\ No newline at end of file
+498	1
diff --git a/sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5
index 326493a1d8cb65c016f9b00f4a47e80cba58c2d2..8bfbbf1e69f57b941cd53fc5ce6c786930f726be 100644
--- a/sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ b/sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5
@@ -306,4 +306,4 @@
 495	1
 496	1
 497	1
-498	1
\ No newline at end of file
+498	1
diff --git a/sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9
index 326493a1d8cb65c016f9b00f4a47e80cba58c2d2..8bfbbf1e69f57b941cd53fc5ce6c786930f726be 100644
--- a/sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9
+++ b/sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9
@@ -306,4 +306,4 @@
 495	1
 496	1
 497	1
-498	1
\ No newline at end of file
+498	1
diff --git a/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e
+++ b/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5
deleted file mode 100644
index 326493a1d8cb65c016f9b00f4a47e80cba58c2d2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ /dev/null
@@ -1,309 +0,0 @@
-0	1
-2	1
-4	1
-5	1
-8	1
-9	1
-10	1
-11	1
-12	1
-15	1
-17	1
-18	1
-19	1
-20	1
-24	1
-26	1
-27	1
-28	1
-30	1
-33	1
-34	1
-35	1
-37	1
-41	1
-42	1
-43	1
-44	1
-47	1
-51	1
-53	1
-54	1
-57	1
-58	1
-64	1
-65	1
-66	1
-67	1
-69	1
-70	1
-72	1
-74	1
-76	1
-77	1
-78	1
-80	1
-82	1
-83	1
-84	1
-85	1
-86	1
-87	1
-90	1
-92	1
-95	1
-96	1
-97	1
-98	1
-100	1
-103	1
-104	1
-105	1
-111	1
-113	1
-114	1
-116	1
-118	1
-119	1
-120	1
-125	1
-126	1
-128	1
-129	1
-131	1
-133	1
-134	1
-136	1
-137	1
-138	1
-143	1
-145	1
-146	1
-149	1
-150	1
-152	1
-153	1
-155	1
-156	1
-157	1
-158	1
-160	1
-162	1
-163	1
-164	1
-165	1
-166	1
-167	1
-168	1
-169	1
-170	1
-172	1
-174	1
-175	1
-176	1
-177	1
-178	1
-179	1
-180	1
-181	1
-183	1
-186	1
-187	1
-189	1
-190	1
-191	1
-192	1
-193	1
-194	1
-195	1
-196	1
-197	1
-199	1
-200	1
-201	1
-202	1
-203	1
-205	1
-207	1
-208	1
-209	1
-213	1
-214	1
-216	1
-217	1
-218	1
-219	1
-221	1
-222	1
-223	1
-224	1
-226	1
-228	1
-229	1
-230	1
-233	1
-235	1
-237	1
-238	1
-239	1
-241	1
-242	1
-244	1
-247	1
-248	1
-249	1
-252	1
-255	1
-256	1
-257	1
-258	1
-260	1
-262	1
-263	1
-265	1
-266	1
-272	1
-273	1
-274	1
-275	1
-277	1
-278	1
-280	1
-281	1
-282	1
-283	1
-284	1
-285	1
-286	1
-287	1
-288	1
-289	1
-291	1
-292	1
-296	1
-298	1
-302	1
-305	1
-306	1
-307	1
-308	1
-309	1
-310	1
-311	1
-315	1
-316	1
-317	1
-318	1
-321	1
-322	1
-323	1
-325	1
-327	1
-331	1
-332	1
-333	1
-335	1
-336	1
-338	1
-339	1
-341	1
-342	1
-344	1
-345	1
-348	1
-351	1
-353	1
-356	1
-360	1
-362	1
-364	1
-365	1
-366	1
-367	1
-368	1
-369	1
-373	1
-374	1
-375	1
-377	1
-378	1
-379	1
-382	1
-384	1
-386	1
-389	1
-392	1
-393	1
-394	1
-395	1
-396	1
-397	1
-399	1
-400	1
-401	1
-402	1
-403	1
-404	1
-406	1
-407	1
-409	1
-411	1
-413	1
-414	1
-417	1
-418	1
-419	1
-421	1
-424	1
-427	1
-429	1
-430	1
-431	1
-432	1
-435	1
-436	1
-437	1
-438	1
-439	1
-443	1
-444	1
-446	1
-448	1
-449	1
-452	1
-453	1
-454	1
-455	1
-457	1
-458	1
-459	1
-460	1
-462	1
-463	1
-466	1
-467	1
-468	1
-469	1
-470	1
-472	1
-475	1
-477	1
-478	1
-479	1
-480	1
-481	1
-482	1
-483	1
-484	1
-485	1
-487	1
-489	1
-490	1
-491	1
-492	1
-493	1
-494	1
-495	1
-496	1
-497	1
-498	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby8_map-7-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby8_map-7-c1a8cf4ef8060a4703b0affe40496169
new file mode 100644
index 0000000000000000000000000000000000000000..8bfbbf1e69f57b941cd53fc5ce6c786930f726be
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby8_map-7-c1a8cf4ef8060a4703b0affe40496169
@@ -0,0 +1,309 @@
+0	1
+2	1
+4	1
+5	1
+8	1
+9	1
+10	1
+11	1
+12	1
+15	1
+17	1
+18	1
+19	1
+20	1
+24	1
+26	1
+27	1
+28	1
+30	1
+33	1
+34	1
+35	1
+37	1
+41	1
+42	1
+43	1
+44	1
+47	1
+51	1
+53	1
+54	1
+57	1
+58	1
+64	1
+65	1
+66	1
+67	1
+69	1
+70	1
+72	1
+74	1
+76	1
+77	1
+78	1
+80	1
+82	1
+83	1
+84	1
+85	1
+86	1
+87	1
+90	1
+92	1
+95	1
+96	1
+97	1
+98	1
+100	1
+103	1
+104	1
+105	1
+111	1
+113	1
+114	1
+116	1
+118	1
+119	1
+120	1
+125	1
+126	1
+128	1
+129	1
+131	1
+133	1
+134	1
+136	1
+137	1
+138	1
+143	1
+145	1
+146	1
+149	1
+150	1
+152	1
+153	1
+155	1
+156	1
+157	1
+158	1
+160	1
+162	1
+163	1
+164	1
+165	1
+166	1
+167	1
+168	1
+169	1
+170	1
+172	1
+174	1
+175	1
+176	1
+177	1
+178	1
+179	1
+180	1
+181	1
+183	1
+186	1
+187	1
+189	1
+190	1
+191	1
+192	1
+193	1
+194	1
+195	1
+196	1
+197	1
+199	1
+200	1
+201	1
+202	1
+203	1
+205	1
+207	1
+208	1
+209	1
+213	1
+214	1
+216	1
+217	1
+218	1
+219	1
+221	1
+222	1
+223	1
+224	1
+226	1
+228	1
+229	1
+230	1
+233	1
+235	1
+237	1
+238	1
+239	1
+241	1
+242	1
+244	1
+247	1
+248	1
+249	1
+252	1
+255	1
+256	1
+257	1
+258	1
+260	1
+262	1
+263	1
+265	1
+266	1
+272	1
+273	1
+274	1
+275	1
+277	1
+278	1
+280	1
+281	1
+282	1
+283	1
+284	1
+285	1
+286	1
+287	1
+288	1
+289	1
+291	1
+292	1
+296	1
+298	1
+302	1
+305	1
+306	1
+307	1
+308	1
+309	1
+310	1
+311	1
+315	1
+316	1
+317	1
+318	1
+321	1
+322	1
+323	1
+325	1
+327	1
+331	1
+332	1
+333	1
+335	1
+336	1
+338	1
+339	1
+341	1
+342	1
+344	1
+345	1
+348	1
+351	1
+353	1
+356	1
+360	1
+362	1
+364	1
+365	1
+366	1
+367	1
+368	1
+369	1
+373	1
+374	1
+375	1
+377	1
+378	1
+379	1
+382	1
+384	1
+386	1
+389	1
+392	1
+393	1
+394	1
+395	1
+396	1
+397	1
+399	1
+400	1
+401	1
+402	1
+403	1
+404	1
+406	1
+407	1
+409	1
+411	1
+413	1
+414	1
+417	1
+418	1
+419	1
+421	1
+424	1
+427	1
+429	1
+430	1
+431	1
+432	1
+435	1
+436	1
+437	1
+438	1
+439	1
+443	1
+444	1
+446	1
+448	1
+449	1
+452	1
+453	1
+454	1
+455	1
+457	1
+458	1
+459	1
+460	1
+462	1
+463	1
+466	1
+467	1
+468	1
+469	1
+470	1
+472	1
+475	1
+477	1
+478	1
+479	1
+480	1
+481	1
+482	1
+483	1
+484	1
+485	1
+487	1
+489	1
+490	1
+491	1
+492	1
+493	1
+494	1
+495	1
+496	1
+497	1
+498	1
diff --git a/sql/hive/src/test/resources/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9
deleted file mode 100644
index 326493a1d8cb65c016f9b00f4a47e80cba58c2d2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9
+++ /dev/null
@@ -1,309 +0,0 @@
-0	1
-2	1
-4	1
-5	1
-8	1
-9	1
-10	1
-11	1
-12	1
-15	1
-17	1
-18	1
-19	1
-20	1
-24	1
-26	1
-27	1
-28	1
-30	1
-33	1
-34	1
-35	1
-37	1
-41	1
-42	1
-43	1
-44	1
-47	1
-51	1
-53	1
-54	1
-57	1
-58	1
-64	1
-65	1
-66	1
-67	1
-69	1
-70	1
-72	1
-74	1
-76	1
-77	1
-78	1
-80	1
-82	1
-83	1
-84	1
-85	1
-86	1
-87	1
-90	1
-92	1
-95	1
-96	1
-97	1
-98	1
-100	1
-103	1
-104	1
-105	1
-111	1
-113	1
-114	1
-116	1
-118	1
-119	1
-120	1
-125	1
-126	1
-128	1
-129	1
-131	1
-133	1
-134	1
-136	1
-137	1
-138	1
-143	1
-145	1
-146	1
-149	1
-150	1
-152	1
-153	1
-155	1
-156	1
-157	1
-158	1
-160	1
-162	1
-163	1
-164	1
-165	1
-166	1
-167	1
-168	1
-169	1
-170	1
-172	1
-174	1
-175	1
-176	1
-177	1
-178	1
-179	1
-180	1
-181	1
-183	1
-186	1
-187	1
-189	1
-190	1
-191	1
-192	1
-193	1
-194	1
-195	1
-196	1
-197	1
-199	1
-200	1
-201	1
-202	1
-203	1
-205	1
-207	1
-208	1
-209	1
-213	1
-214	1
-216	1
-217	1
-218	1
-219	1
-221	1
-222	1
-223	1
-224	1
-226	1
-228	1
-229	1
-230	1
-233	1
-235	1
-237	1
-238	1
-239	1
-241	1
-242	1
-244	1
-247	1
-248	1
-249	1
-252	1
-255	1
-256	1
-257	1
-258	1
-260	1
-262	1
-263	1
-265	1
-266	1
-272	1
-273	1
-274	1
-275	1
-277	1
-278	1
-280	1
-281	1
-282	1
-283	1
-284	1
-285	1
-286	1
-287	1
-288	1
-289	1
-291	1
-292	1
-296	1
-298	1
-302	1
-305	1
-306	1
-307	1
-308	1
-309	1
-310	1
-311	1
-315	1
-316	1
-317	1
-318	1
-321	1
-322	1
-323	1
-325	1
-327	1
-331	1
-332	1
-333	1
-335	1
-336	1
-338	1
-339	1
-341	1
-342	1
-344	1
-345	1
-348	1
-351	1
-353	1
-356	1
-360	1
-362	1
-364	1
-365	1
-366	1
-367	1
-368	1
-369	1
-373	1
-374	1
-375	1
-377	1
-378	1
-379	1
-382	1
-384	1
-386	1
-389	1
-392	1
-393	1
-394	1
-395	1
-396	1
-397	1
-399	1
-400	1
-401	1
-402	1
-403	1
-404	1
-406	1
-407	1
-409	1
-411	1
-413	1
-414	1
-417	1
-418	1
-419	1
-421	1
-424	1
-427	1
-429	1
-430	1
-431	1
-432	1
-435	1
-436	1
-437	1
-438	1
-439	1
-443	1
-444	1
-446	1
-448	1
-449	1
-452	1
-453	1
-454	1
-455	1
-457	1
-458	1
-459	1
-460	1
-462	1
-463	1
-466	1
-467	1
-468	1
-469	1
-470	1
-472	1
-475	1
-477	1
-478	1
-479	1
-480	1
-481	1
-482	1
-483	1
-484	1
-485	1
-487	1
-489	1
-490	1
-491	1
-492	1
-493	1
-494	1
-495	1
-496	1
-497	1
-498	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby8_map-8-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby8_map-8-6c26c5d39c4bdb61728defa9b44bfb52
new file mode 100644
index 0000000000000000000000000000000000000000..8bfbbf1e69f57b941cd53fc5ce6c786930f726be
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby8_map-8-6c26c5d39c4bdb61728defa9b44bfb52
@@ -0,0 +1,309 @@
+0	1
+2	1
+4	1
+5	1
+8	1
+9	1
+10	1
+11	1
+12	1
+15	1
+17	1
+18	1
+19	1
+20	1
+24	1
+26	1
+27	1
+28	1
+30	1
+33	1
+34	1
+35	1
+37	1
+41	1
+42	1
+43	1
+44	1
+47	1
+51	1
+53	1
+54	1
+57	1
+58	1
+64	1
+65	1
+66	1
+67	1
+69	1
+70	1
+72	1
+74	1
+76	1
+77	1
+78	1
+80	1
+82	1
+83	1
+84	1
+85	1
+86	1
+87	1
+90	1
+92	1
+95	1
+96	1
+97	1
+98	1
+100	1
+103	1
+104	1
+105	1
+111	1
+113	1
+114	1
+116	1
+118	1
+119	1
+120	1
+125	1
+126	1
+128	1
+129	1
+131	1
+133	1
+134	1
+136	1
+137	1
+138	1
+143	1
+145	1
+146	1
+149	1
+150	1
+152	1
+153	1
+155	1
+156	1
+157	1
+158	1
+160	1
+162	1
+163	1
+164	1
+165	1
+166	1
+167	1
+168	1
+169	1
+170	1
+172	1
+174	1
+175	1
+176	1
+177	1
+178	1
+179	1
+180	1
+181	1
+183	1
+186	1
+187	1
+189	1
+190	1
+191	1
+192	1
+193	1
+194	1
+195	1
+196	1
+197	1
+199	1
+200	1
+201	1
+202	1
+203	1
+205	1
+207	1
+208	1
+209	1
+213	1
+214	1
+216	1
+217	1
+218	1
+219	1
+221	1
+222	1
+223	1
+224	1
+226	1
+228	1
+229	1
+230	1
+233	1
+235	1
+237	1
+238	1
+239	1
+241	1
+242	1
+244	1
+247	1
+248	1
+249	1
+252	1
+255	1
+256	1
+257	1
+258	1
+260	1
+262	1
+263	1
+265	1
+266	1
+272	1
+273	1
+274	1
+275	1
+277	1
+278	1
+280	1
+281	1
+282	1
+283	1
+284	1
+285	1
+286	1
+287	1
+288	1
+289	1
+291	1
+292	1
+296	1
+298	1
+302	1
+305	1
+306	1
+307	1
+308	1
+309	1
+310	1
+311	1
+315	1
+316	1
+317	1
+318	1
+321	1
+322	1
+323	1
+325	1
+327	1
+331	1
+332	1
+333	1
+335	1
+336	1
+338	1
+339	1
+341	1
+342	1
+344	1
+345	1
+348	1
+351	1
+353	1
+356	1
+360	1
+362	1
+364	1
+365	1
+366	1
+367	1
+368	1
+369	1
+373	1
+374	1
+375	1
+377	1
+378	1
+379	1
+382	1
+384	1
+386	1
+389	1
+392	1
+393	1
+394	1
+395	1
+396	1
+397	1
+399	1
+400	1
+401	1
+402	1
+403	1
+404	1
+406	1
+407	1
+409	1
+411	1
+413	1
+414	1
+417	1
+418	1
+419	1
+421	1
+424	1
+427	1
+429	1
+430	1
+431	1
+432	1
+435	1
+436	1
+437	1
+438	1
+439	1
+443	1
+444	1
+446	1
+448	1
+449	1
+452	1
+453	1
+454	1
+455	1
+457	1
+458	1
+459	1
+460	1
+462	1
+463	1
+466	1
+467	1
+468	1
+469	1
+470	1
+472	1
+475	1
+477	1
+478	1
+479	1
+480	1
+481	1
+482	1
+483	1
+484	1
+485	1
+487	1
+489	1
+490	1
+491	1
+492	1
+493	1
+494	1
+495	1
+496	1
+497	1
+498	1
diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5
deleted file mode 100644
index 326493a1d8cb65c016f9b00f4a47e80cba58c2d2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ /dev/null
@@ -1,309 +0,0 @@
-0	1
-2	1
-4	1
-5	1
-8	1
-9	1
-10	1
-11	1
-12	1
-15	1
-17	1
-18	1
-19	1
-20	1
-24	1
-26	1
-27	1
-28	1
-30	1
-33	1
-34	1
-35	1
-37	1
-41	1
-42	1
-43	1
-44	1
-47	1
-51	1
-53	1
-54	1
-57	1
-58	1
-64	1
-65	1
-66	1
-67	1
-69	1
-70	1
-72	1
-74	1
-76	1
-77	1
-78	1
-80	1
-82	1
-83	1
-84	1
-85	1
-86	1
-87	1
-90	1
-92	1
-95	1
-96	1
-97	1
-98	1
-100	1
-103	1
-104	1
-105	1
-111	1
-113	1
-114	1
-116	1
-118	1
-119	1
-120	1
-125	1
-126	1
-128	1
-129	1
-131	1
-133	1
-134	1
-136	1
-137	1
-138	1
-143	1
-145	1
-146	1
-149	1
-150	1
-152	1
-153	1
-155	1
-156	1
-157	1
-158	1
-160	1
-162	1
-163	1
-164	1
-165	1
-166	1
-167	1
-168	1
-169	1
-170	1
-172	1
-174	1
-175	1
-176	1
-177	1
-178	1
-179	1
-180	1
-181	1
-183	1
-186	1
-187	1
-189	1
-190	1
-191	1
-192	1
-193	1
-194	1
-195	1
-196	1
-197	1
-199	1
-200	1
-201	1
-202	1
-203	1
-205	1
-207	1
-208	1
-209	1
-213	1
-214	1
-216	1
-217	1
-218	1
-219	1
-221	1
-222	1
-223	1
-224	1
-226	1
-228	1
-229	1
-230	1
-233	1
-235	1
-237	1
-238	1
-239	1
-241	1
-242	1
-244	1
-247	1
-248	1
-249	1
-252	1
-255	1
-256	1
-257	1
-258	1
-260	1
-262	1
-263	1
-265	1
-266	1
-272	1
-273	1
-274	1
-275	1
-277	1
-278	1
-280	1
-281	1
-282	1
-283	1
-284	1
-285	1
-286	1
-287	1
-288	1
-289	1
-291	1
-292	1
-296	1
-298	1
-302	1
-305	1
-306	1
-307	1
-308	1
-309	1
-310	1
-311	1
-315	1
-316	1
-317	1
-318	1
-321	1
-322	1
-323	1
-325	1
-327	1
-331	1
-332	1
-333	1
-335	1
-336	1
-338	1
-339	1
-341	1
-342	1
-344	1
-345	1
-348	1
-351	1
-353	1
-356	1
-360	1
-362	1
-364	1
-365	1
-366	1
-367	1
-368	1
-369	1
-373	1
-374	1
-375	1
-377	1
-378	1
-379	1
-382	1
-384	1
-386	1
-389	1
-392	1
-393	1
-394	1
-395	1
-396	1
-397	1
-399	1
-400	1
-401	1
-402	1
-403	1
-404	1
-406	1
-407	1
-409	1
-411	1
-413	1
-414	1
-417	1
-418	1
-419	1
-421	1
-424	1
-427	1
-429	1
-430	1
-431	1
-432	1
-435	1
-436	1
-437	1
-438	1
-439	1
-443	1
-444	1
-446	1
-448	1
-449	1
-452	1
-453	1
-454	1
-455	1
-457	1
-458	1
-459	1
-460	1
-462	1
-463	1
-466	1
-467	1
-468	1
-469	1
-470	1
-472	1
-475	1
-477	1
-478	1
-479	1
-480	1
-481	1
-482	1
-483	1
-484	1
-485	1
-487	1
-489	1
-490	1
-491	1
-492	1
-493	1
-494	1
-495	1
-496	1
-497	1
-498	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-7-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby8_map_skew-7-c1a8cf4ef8060a4703b0affe40496169
new file mode 100644
index 0000000000000000000000000000000000000000..8bfbbf1e69f57b941cd53fc5ce6c786930f726be
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-7-c1a8cf4ef8060a4703b0affe40496169
@@ -0,0 +1,309 @@
+0	1
+2	1
+4	1
+5	1
+8	1
+9	1
+10	1
+11	1
+12	1
+15	1
+17	1
+18	1
+19	1
+20	1
+24	1
+26	1
+27	1
+28	1
+30	1
+33	1
+34	1
+35	1
+37	1
+41	1
+42	1
+43	1
+44	1
+47	1
+51	1
+53	1
+54	1
+57	1
+58	1
+64	1
+65	1
+66	1
+67	1
+69	1
+70	1
+72	1
+74	1
+76	1
+77	1
+78	1
+80	1
+82	1
+83	1
+84	1
+85	1
+86	1
+87	1
+90	1
+92	1
+95	1
+96	1
+97	1
+98	1
+100	1
+103	1
+104	1
+105	1
+111	1
+113	1
+114	1
+116	1
+118	1
+119	1
+120	1
+125	1
+126	1
+128	1
+129	1
+131	1
+133	1
+134	1
+136	1
+137	1
+138	1
+143	1
+145	1
+146	1
+149	1
+150	1
+152	1
+153	1
+155	1
+156	1
+157	1
+158	1
+160	1
+162	1
+163	1
+164	1
+165	1
+166	1
+167	1
+168	1
+169	1
+170	1
+172	1
+174	1
+175	1
+176	1
+177	1
+178	1
+179	1
+180	1
+181	1
+183	1
+186	1
+187	1
+189	1
+190	1
+191	1
+192	1
+193	1
+194	1
+195	1
+196	1
+197	1
+199	1
+200	1
+201	1
+202	1
+203	1
+205	1
+207	1
+208	1
+209	1
+213	1
+214	1
+216	1
+217	1
+218	1
+219	1
+221	1
+222	1
+223	1
+224	1
+226	1
+228	1
+229	1
+230	1
+233	1
+235	1
+237	1
+238	1
+239	1
+241	1
+242	1
+244	1
+247	1
+248	1
+249	1
+252	1
+255	1
+256	1
+257	1
+258	1
+260	1
+262	1
+263	1
+265	1
+266	1
+272	1
+273	1
+274	1
+275	1
+277	1
+278	1
+280	1
+281	1
+282	1
+283	1
+284	1
+285	1
+286	1
+287	1
+288	1
+289	1
+291	1
+292	1
+296	1
+298	1
+302	1
+305	1
+306	1
+307	1
+308	1
+309	1
+310	1
+311	1
+315	1
+316	1
+317	1
+318	1
+321	1
+322	1
+323	1
+325	1
+327	1
+331	1
+332	1
+333	1
+335	1
+336	1
+338	1
+339	1
+341	1
+342	1
+344	1
+345	1
+348	1
+351	1
+353	1
+356	1
+360	1
+362	1
+364	1
+365	1
+366	1
+367	1
+368	1
+369	1
+373	1
+374	1
+375	1
+377	1
+378	1
+379	1
+382	1
+384	1
+386	1
+389	1
+392	1
+393	1
+394	1
+395	1
+396	1
+397	1
+399	1
+400	1
+401	1
+402	1
+403	1
+404	1
+406	1
+407	1
+409	1
+411	1
+413	1
+414	1
+417	1
+418	1
+419	1
+421	1
+424	1
+427	1
+429	1
+430	1
+431	1
+432	1
+435	1
+436	1
+437	1
+438	1
+439	1
+443	1
+444	1
+446	1
+448	1
+449	1
+452	1
+453	1
+454	1
+455	1
+457	1
+458	1
+459	1
+460	1
+462	1
+463	1
+466	1
+467	1
+468	1
+469	1
+470	1
+472	1
+475	1
+477	1
+478	1
+479	1
+480	1
+481	1
+482	1
+483	1
+484	1
+485	1
+487	1
+489	1
+490	1
+491	1
+492	1
+493	1
+494	1
+495	1
+496	1
+497	1
+498	1
diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9
deleted file mode 100644
index 326493a1d8cb65c016f9b00f4a47e80cba58c2d2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9
+++ /dev/null
@@ -1,309 +0,0 @@
-0	1
-2	1
-4	1
-5	1
-8	1
-9	1
-10	1
-11	1
-12	1
-15	1
-17	1
-18	1
-19	1
-20	1
-24	1
-26	1
-27	1
-28	1
-30	1
-33	1
-34	1
-35	1
-37	1
-41	1
-42	1
-43	1
-44	1
-47	1
-51	1
-53	1
-54	1
-57	1
-58	1
-64	1
-65	1
-66	1
-67	1
-69	1
-70	1
-72	1
-74	1
-76	1
-77	1
-78	1
-80	1
-82	1
-83	1
-84	1
-85	1
-86	1
-87	1
-90	1
-92	1
-95	1
-96	1
-97	1
-98	1
-100	1
-103	1
-104	1
-105	1
-111	1
-113	1
-114	1
-116	1
-118	1
-119	1
-120	1
-125	1
-126	1
-128	1
-129	1
-131	1
-133	1
-134	1
-136	1
-137	1
-138	1
-143	1
-145	1
-146	1
-149	1
-150	1
-152	1
-153	1
-155	1
-156	1
-157	1
-158	1
-160	1
-162	1
-163	1
-164	1
-165	1
-166	1
-167	1
-168	1
-169	1
-170	1
-172	1
-174	1
-175	1
-176	1
-177	1
-178	1
-179	1
-180	1
-181	1
-183	1
-186	1
-187	1
-189	1
-190	1
-191	1
-192	1
-193	1
-194	1
-195	1
-196	1
-197	1
-199	1
-200	1
-201	1
-202	1
-203	1
-205	1
-207	1
-208	1
-209	1
-213	1
-214	1
-216	1
-217	1
-218	1
-219	1
-221	1
-222	1
-223	1
-224	1
-226	1
-228	1
-229	1
-230	1
-233	1
-235	1
-237	1
-238	1
-239	1
-241	1
-242	1
-244	1
-247	1
-248	1
-249	1
-252	1
-255	1
-256	1
-257	1
-258	1
-260	1
-262	1
-263	1
-265	1
-266	1
-272	1
-273	1
-274	1
-275	1
-277	1
-278	1
-280	1
-281	1
-282	1
-283	1
-284	1
-285	1
-286	1
-287	1
-288	1
-289	1
-291	1
-292	1
-296	1
-298	1
-302	1
-305	1
-306	1
-307	1
-308	1
-309	1
-310	1
-311	1
-315	1
-316	1
-317	1
-318	1
-321	1
-322	1
-323	1
-325	1
-327	1
-331	1
-332	1
-333	1
-335	1
-336	1
-338	1
-339	1
-341	1
-342	1
-344	1
-345	1
-348	1
-351	1
-353	1
-356	1
-360	1
-362	1
-364	1
-365	1
-366	1
-367	1
-368	1
-369	1
-373	1
-374	1
-375	1
-377	1
-378	1
-379	1
-382	1
-384	1
-386	1
-389	1
-392	1
-393	1
-394	1
-395	1
-396	1
-397	1
-399	1
-400	1
-401	1
-402	1
-403	1
-404	1
-406	1
-407	1
-409	1
-411	1
-413	1
-414	1
-417	1
-418	1
-419	1
-421	1
-424	1
-427	1
-429	1
-430	1
-431	1
-432	1
-435	1
-436	1
-437	1
-438	1
-439	1
-443	1
-444	1
-446	1
-448	1
-449	1
-452	1
-453	1
-454	1
-455	1
-457	1
-458	1
-459	1
-460	1
-462	1
-463	1
-466	1
-467	1
-468	1
-469	1
-470	1
-472	1
-475	1
-477	1
-478	1
-479	1
-480	1
-481	1
-482	1
-483	1
-484	1
-485	1
-487	1
-489	1
-490	1
-491	1
-492	1
-493	1
-494	1
-495	1
-496	1
-497	1
-498	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-8-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby8_map_skew-8-6c26c5d39c4bdb61728defa9b44bfb52
new file mode 100644
index 0000000000000000000000000000000000000000..8bfbbf1e69f57b941cd53fc5ce6c786930f726be
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-8-6c26c5d39c4bdb61728defa9b44bfb52
@@ -0,0 +1,309 @@
+0	1
+2	1
+4	1
+5	1
+8	1
+9	1
+10	1
+11	1
+12	1
+15	1
+17	1
+18	1
+19	1
+20	1
+24	1
+26	1
+27	1
+28	1
+30	1
+33	1
+34	1
+35	1
+37	1
+41	1
+42	1
+43	1
+44	1
+47	1
+51	1
+53	1
+54	1
+57	1
+58	1
+64	1
+65	1
+66	1
+67	1
+69	1
+70	1
+72	1
+74	1
+76	1
+77	1
+78	1
+80	1
+82	1
+83	1
+84	1
+85	1
+86	1
+87	1
+90	1
+92	1
+95	1
+96	1
+97	1
+98	1
+100	1
+103	1
+104	1
+105	1
+111	1
+113	1
+114	1
+116	1
+118	1
+119	1
+120	1
+125	1
+126	1
+128	1
+129	1
+131	1
+133	1
+134	1
+136	1
+137	1
+138	1
+143	1
+145	1
+146	1
+149	1
+150	1
+152	1
+153	1
+155	1
+156	1
+157	1
+158	1
+160	1
+162	1
+163	1
+164	1
+165	1
+166	1
+167	1
+168	1
+169	1
+170	1
+172	1
+174	1
+175	1
+176	1
+177	1
+178	1
+179	1
+180	1
+181	1
+183	1
+186	1
+187	1
+189	1
+190	1
+191	1
+192	1
+193	1
+194	1
+195	1
+196	1
+197	1
+199	1
+200	1
+201	1
+202	1
+203	1
+205	1
+207	1
+208	1
+209	1
+213	1
+214	1
+216	1
+217	1
+218	1
+219	1
+221	1
+222	1
+223	1
+224	1
+226	1
+228	1
+229	1
+230	1
+233	1
+235	1
+237	1
+238	1
+239	1
+241	1
+242	1
+244	1
+247	1
+248	1
+249	1
+252	1
+255	1
+256	1
+257	1
+258	1
+260	1
+262	1
+263	1
+265	1
+266	1
+272	1
+273	1
+274	1
+275	1
+277	1
+278	1
+280	1
+281	1
+282	1
+283	1
+284	1
+285	1
+286	1
+287	1
+288	1
+289	1
+291	1
+292	1
+296	1
+298	1
+302	1
+305	1
+306	1
+307	1
+308	1
+309	1
+310	1
+311	1
+315	1
+316	1
+317	1
+318	1
+321	1
+322	1
+323	1
+325	1
+327	1
+331	1
+332	1
+333	1
+335	1
+336	1
+338	1
+339	1
+341	1
+342	1
+344	1
+345	1
+348	1
+351	1
+353	1
+356	1
+360	1
+362	1
+364	1
+365	1
+366	1
+367	1
+368	1
+369	1
+373	1
+374	1
+375	1
+377	1
+378	1
+379	1
+382	1
+384	1
+386	1
+389	1
+392	1
+393	1
+394	1
+395	1
+396	1
+397	1
+399	1
+400	1
+401	1
+402	1
+403	1
+404	1
+406	1
+407	1
+409	1
+411	1
+413	1
+414	1
+417	1
+418	1
+419	1
+421	1
+424	1
+427	1
+429	1
+430	1
+431	1
+432	1
+435	1
+436	1
+437	1
+438	1
+439	1
+443	1
+444	1
+446	1
+448	1
+449	1
+452	1
+453	1
+454	1
+455	1
+457	1
+458	1
+459	1
+460	1
+462	1
+463	1
+466	1
+467	1
+468	1
+469	1
+470	1
+472	1
+475	1
+477	1
+478	1
+479	1
+480	1
+481	1
+482	1
+483	1
+484	1
+485	1
+487	1
+489	1
+490	1
+491	1
+492	1
+493	1
+494	1
+495	1
+496	1
+497	1
+498	1
diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5
deleted file mode 100644
index 326493a1d8cb65c016f9b00f4a47e80cba58c2d2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ /dev/null
@@ -1,309 +0,0 @@
-0	1
-2	1
-4	1
-5	1
-8	1
-9	1
-10	1
-11	1
-12	1
-15	1
-17	1
-18	1
-19	1
-20	1
-24	1
-26	1
-27	1
-28	1
-30	1
-33	1
-34	1
-35	1
-37	1
-41	1
-42	1
-43	1
-44	1
-47	1
-51	1
-53	1
-54	1
-57	1
-58	1
-64	1
-65	1
-66	1
-67	1
-69	1
-70	1
-72	1
-74	1
-76	1
-77	1
-78	1
-80	1
-82	1
-83	1
-84	1
-85	1
-86	1
-87	1
-90	1
-92	1
-95	1
-96	1
-97	1
-98	1
-100	1
-103	1
-104	1
-105	1
-111	1
-113	1
-114	1
-116	1
-118	1
-119	1
-120	1
-125	1
-126	1
-128	1
-129	1
-131	1
-133	1
-134	1
-136	1
-137	1
-138	1
-143	1
-145	1
-146	1
-149	1
-150	1
-152	1
-153	1
-155	1
-156	1
-157	1
-158	1
-160	1
-162	1
-163	1
-164	1
-165	1
-166	1
-167	1
-168	1
-169	1
-170	1
-172	1
-174	1
-175	1
-176	1
-177	1
-178	1
-179	1
-180	1
-181	1
-183	1
-186	1
-187	1
-189	1
-190	1
-191	1
-192	1
-193	1
-194	1
-195	1
-196	1
-197	1
-199	1
-200	1
-201	1
-202	1
-203	1
-205	1
-207	1
-208	1
-209	1
-213	1
-214	1
-216	1
-217	1
-218	1
-219	1
-221	1
-222	1
-223	1
-224	1
-226	1
-228	1
-229	1
-230	1
-233	1
-235	1
-237	1
-238	1
-239	1
-241	1
-242	1
-244	1
-247	1
-248	1
-249	1
-252	1
-255	1
-256	1
-257	1
-258	1
-260	1
-262	1
-263	1
-265	1
-266	1
-272	1
-273	1
-274	1
-275	1
-277	1
-278	1
-280	1
-281	1
-282	1
-283	1
-284	1
-285	1
-286	1
-287	1
-288	1
-289	1
-291	1
-292	1
-296	1
-298	1
-302	1
-305	1
-306	1
-307	1
-308	1
-309	1
-310	1
-311	1
-315	1
-316	1
-317	1
-318	1
-321	1
-322	1
-323	1
-325	1
-327	1
-331	1
-332	1
-333	1
-335	1
-336	1
-338	1
-339	1
-341	1
-342	1
-344	1
-345	1
-348	1
-351	1
-353	1
-356	1
-360	1
-362	1
-364	1
-365	1
-366	1
-367	1
-368	1
-369	1
-373	1
-374	1
-375	1
-377	1
-378	1
-379	1
-382	1
-384	1
-386	1
-389	1
-392	1
-393	1
-394	1
-395	1
-396	1
-397	1
-399	1
-400	1
-401	1
-402	1
-403	1
-404	1
-406	1
-407	1
-409	1
-411	1
-413	1
-414	1
-417	1
-418	1
-419	1
-421	1
-424	1
-427	1
-429	1
-430	1
-431	1
-432	1
-435	1
-436	1
-437	1
-438	1
-439	1
-443	1
-444	1
-446	1
-448	1
-449	1
-452	1
-453	1
-454	1
-455	1
-457	1
-458	1
-459	1
-460	1
-462	1
-463	1
-466	1
-467	1
-468	1
-469	1
-470	1
-472	1
-475	1
-477	1
-478	1
-479	1
-480	1
-481	1
-482	1
-483	1
-484	1
-485	1
-487	1
-489	1
-490	1
-491	1
-492	1
-493	1
-494	1
-495	1
-496	1
-497	1
-498	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-7-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby8_noskew-7-c1a8cf4ef8060a4703b0affe40496169
new file mode 100644
index 0000000000000000000000000000000000000000..8bfbbf1e69f57b941cd53fc5ce6c786930f726be
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby8_noskew-7-c1a8cf4ef8060a4703b0affe40496169
@@ -0,0 +1,309 @@
+0	1
+2	1
+4	1
+5	1
+8	1
+9	1
+10	1
+11	1
+12	1
+15	1
+17	1
+18	1
+19	1
+20	1
+24	1
+26	1
+27	1
+28	1
+30	1
+33	1
+34	1
+35	1
+37	1
+41	1
+42	1
+43	1
+44	1
+47	1
+51	1
+53	1
+54	1
+57	1
+58	1
+64	1
+65	1
+66	1
+67	1
+69	1
+70	1
+72	1
+74	1
+76	1
+77	1
+78	1
+80	1
+82	1
+83	1
+84	1
+85	1
+86	1
+87	1
+90	1
+92	1
+95	1
+96	1
+97	1
+98	1
+100	1
+103	1
+104	1
+105	1
+111	1
+113	1
+114	1
+116	1
+118	1
+119	1
+120	1
+125	1
+126	1
+128	1
+129	1
+131	1
+133	1
+134	1
+136	1
+137	1
+138	1
+143	1
+145	1
+146	1
+149	1
+150	1
+152	1
+153	1
+155	1
+156	1
+157	1
+158	1
+160	1
+162	1
+163	1
+164	1
+165	1
+166	1
+167	1
+168	1
+169	1
+170	1
+172	1
+174	1
+175	1
+176	1
+177	1
+178	1
+179	1
+180	1
+181	1
+183	1
+186	1
+187	1
+189	1
+190	1
+191	1
+192	1
+193	1
+194	1
+195	1
+196	1
+197	1
+199	1
+200	1
+201	1
+202	1
+203	1
+205	1
+207	1
+208	1
+209	1
+213	1
+214	1
+216	1
+217	1
+218	1
+219	1
+221	1
+222	1
+223	1
+224	1
+226	1
+228	1
+229	1
+230	1
+233	1
+235	1
+237	1
+238	1
+239	1
+241	1
+242	1
+244	1
+247	1
+248	1
+249	1
+252	1
+255	1
+256	1
+257	1
+258	1
+260	1
+262	1
+263	1
+265	1
+266	1
+272	1
+273	1
+274	1
+275	1
+277	1
+278	1
+280	1
+281	1
+282	1
+283	1
+284	1
+285	1
+286	1
+287	1
+288	1
+289	1
+291	1
+292	1
+296	1
+298	1
+302	1
+305	1
+306	1
+307	1
+308	1
+309	1
+310	1
+311	1
+315	1
+316	1
+317	1
+318	1
+321	1
+322	1
+323	1
+325	1
+327	1
+331	1
+332	1
+333	1
+335	1
+336	1
+338	1
+339	1
+341	1
+342	1
+344	1
+345	1
+348	1
+351	1
+353	1
+356	1
+360	1
+362	1
+364	1
+365	1
+366	1
+367	1
+368	1
+369	1
+373	1
+374	1
+375	1
+377	1
+378	1
+379	1
+382	1
+384	1
+386	1
+389	1
+392	1
+393	1
+394	1
+395	1
+396	1
+397	1
+399	1
+400	1
+401	1
+402	1
+403	1
+404	1
+406	1
+407	1
+409	1
+411	1
+413	1
+414	1
+417	1
+418	1
+419	1
+421	1
+424	1
+427	1
+429	1
+430	1
+431	1
+432	1
+435	1
+436	1
+437	1
+438	1
+439	1
+443	1
+444	1
+446	1
+448	1
+449	1
+452	1
+453	1
+454	1
+455	1
+457	1
+458	1
+459	1
+460	1
+462	1
+463	1
+466	1
+467	1
+468	1
+469	1
+470	1
+472	1
+475	1
+477	1
+478	1
+479	1
+480	1
+481	1
+482	1
+483	1
+484	1
+485	1
+487	1
+489	1
+490	1
+491	1
+492	1
+493	1
+494	1
+495	1
+496	1
+497	1
+498	1
diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9
deleted file mode 100644
index 326493a1d8cb65c016f9b00f4a47e80cba58c2d2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9
+++ /dev/null
@@ -1,309 +0,0 @@
-0	1
-2	1
-4	1
-5	1
-8	1
-9	1
-10	1
-11	1
-12	1
-15	1
-17	1
-18	1
-19	1
-20	1
-24	1
-26	1
-27	1
-28	1
-30	1
-33	1
-34	1
-35	1
-37	1
-41	1
-42	1
-43	1
-44	1
-47	1
-51	1
-53	1
-54	1
-57	1
-58	1
-64	1
-65	1
-66	1
-67	1
-69	1
-70	1
-72	1
-74	1
-76	1
-77	1
-78	1
-80	1
-82	1
-83	1
-84	1
-85	1
-86	1
-87	1
-90	1
-92	1
-95	1
-96	1
-97	1
-98	1
-100	1
-103	1
-104	1
-105	1
-111	1
-113	1
-114	1
-116	1
-118	1
-119	1
-120	1
-125	1
-126	1
-128	1
-129	1
-131	1
-133	1
-134	1
-136	1
-137	1
-138	1
-143	1
-145	1
-146	1
-149	1
-150	1
-152	1
-153	1
-155	1
-156	1
-157	1
-158	1
-160	1
-162	1
-163	1
-164	1
-165	1
-166	1
-167	1
-168	1
-169	1
-170	1
-172	1
-174	1
-175	1
-176	1
-177	1
-178	1
-179	1
-180	1
-181	1
-183	1
-186	1
-187	1
-189	1
-190	1
-191	1
-192	1
-193	1
-194	1
-195	1
-196	1
-197	1
-199	1
-200	1
-201	1
-202	1
-203	1
-205	1
-207	1
-208	1
-209	1
-213	1
-214	1
-216	1
-217	1
-218	1
-219	1
-221	1
-222	1
-223	1
-224	1
-226	1
-228	1
-229	1
-230	1
-233	1
-235	1
-237	1
-238	1
-239	1
-241	1
-242	1
-244	1
-247	1
-248	1
-249	1
-252	1
-255	1
-256	1
-257	1
-258	1
-260	1
-262	1
-263	1
-265	1
-266	1
-272	1
-273	1
-274	1
-275	1
-277	1
-278	1
-280	1
-281	1
-282	1
-283	1
-284	1
-285	1
-286	1
-287	1
-288	1
-289	1
-291	1
-292	1
-296	1
-298	1
-302	1
-305	1
-306	1
-307	1
-308	1
-309	1
-310	1
-311	1
-315	1
-316	1
-317	1
-318	1
-321	1
-322	1
-323	1
-325	1
-327	1
-331	1
-332	1
-333	1
-335	1
-336	1
-338	1
-339	1
-341	1
-342	1
-344	1
-345	1
-348	1
-351	1
-353	1
-356	1
-360	1
-362	1
-364	1
-365	1
-366	1
-367	1
-368	1
-369	1
-373	1
-374	1
-375	1
-377	1
-378	1
-379	1
-382	1
-384	1
-386	1
-389	1
-392	1
-393	1
-394	1
-395	1
-396	1
-397	1
-399	1
-400	1
-401	1
-402	1
-403	1
-404	1
-406	1
-407	1
-409	1
-411	1
-413	1
-414	1
-417	1
-418	1
-419	1
-421	1
-424	1
-427	1
-429	1
-430	1
-431	1
-432	1
-435	1
-436	1
-437	1
-438	1
-439	1
-443	1
-444	1
-446	1
-448	1
-449	1
-452	1
-453	1
-454	1
-455	1
-457	1
-458	1
-459	1
-460	1
-462	1
-463	1
-466	1
-467	1
-468	1
-469	1
-470	1
-472	1
-475	1
-477	1
-478	1
-479	1
-480	1
-481	1
-482	1
-483	1
-484	1
-485	1
-487	1
-489	1
-490	1
-491	1
-492	1
-493	1
-494	1
-495	1
-496	1
-497	1
-498	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-8-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby8_noskew-8-6c26c5d39c4bdb61728defa9b44bfb52
new file mode 100644
index 0000000000000000000000000000000000000000..8bfbbf1e69f57b941cd53fc5ce6c786930f726be
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby8_noskew-8-6c26c5d39c4bdb61728defa9b44bfb52
@@ -0,0 +1,309 @@
+0	1
+2	1
+4	1
+5	1
+8	1
+9	1
+10	1
+11	1
+12	1
+15	1
+17	1
+18	1
+19	1
+20	1
+24	1
+26	1
+27	1
+28	1
+30	1
+33	1
+34	1
+35	1
+37	1
+41	1
+42	1
+43	1
+44	1
+47	1
+51	1
+53	1
+54	1
+57	1
+58	1
+64	1
+65	1
+66	1
+67	1
+69	1
+70	1
+72	1
+74	1
+76	1
+77	1
+78	1
+80	1
+82	1
+83	1
+84	1
+85	1
+86	1
+87	1
+90	1
+92	1
+95	1
+96	1
+97	1
+98	1
+100	1
+103	1
+104	1
+105	1
+111	1
+113	1
+114	1
+116	1
+118	1
+119	1
+120	1
+125	1
+126	1
+128	1
+129	1
+131	1
+133	1
+134	1
+136	1
+137	1
+138	1
+143	1
+145	1
+146	1
+149	1
+150	1
+152	1
+153	1
+155	1
+156	1
+157	1
+158	1
+160	1
+162	1
+163	1
+164	1
+165	1
+166	1
+167	1
+168	1
+169	1
+170	1
+172	1
+174	1
+175	1
+176	1
+177	1
+178	1
+179	1
+180	1
+181	1
+183	1
+186	1
+187	1
+189	1
+190	1
+191	1
+192	1
+193	1
+194	1
+195	1
+196	1
+197	1
+199	1
+200	1
+201	1
+202	1
+203	1
+205	1
+207	1
+208	1
+209	1
+213	1
+214	1
+216	1
+217	1
+218	1
+219	1
+221	1
+222	1
+223	1
+224	1
+226	1
+228	1
+229	1
+230	1
+233	1
+235	1
+237	1
+238	1
+239	1
+241	1
+242	1
+244	1
+247	1
+248	1
+249	1
+252	1
+255	1
+256	1
+257	1
+258	1
+260	1
+262	1
+263	1
+265	1
+266	1
+272	1
+273	1
+274	1
+275	1
+277	1
+278	1
+280	1
+281	1
+282	1
+283	1
+284	1
+285	1
+286	1
+287	1
+288	1
+289	1
+291	1
+292	1
+296	1
+298	1
+302	1
+305	1
+306	1
+307	1
+308	1
+309	1
+310	1
+311	1
+315	1
+316	1
+317	1
+318	1
+321	1
+322	1
+323	1
+325	1
+327	1
+331	1
+332	1
+333	1
+335	1
+336	1
+338	1
+339	1
+341	1
+342	1
+344	1
+345	1
+348	1
+351	1
+353	1
+356	1
+360	1
+362	1
+364	1
+365	1
+366	1
+367	1
+368	1
+369	1
+373	1
+374	1
+375	1
+377	1
+378	1
+379	1
+382	1
+384	1
+386	1
+389	1
+392	1
+393	1
+394	1
+395	1
+396	1
+397	1
+399	1
+400	1
+401	1
+402	1
+403	1
+404	1
+406	1
+407	1
+409	1
+411	1
+413	1
+414	1
+417	1
+418	1
+419	1
+421	1
+424	1
+427	1
+429	1
+430	1
+431	1
+432	1
+435	1
+436	1
+437	1
+438	1
+439	1
+443	1
+444	1
+446	1
+448	1
+449	1
+452	1
+453	1
+454	1
+455	1
+457	1
+458	1
+459	1
+460	1
+462	1
+463	1
+466	1
+467	1
+468	1
+469	1
+470	1
+472	1
+475	1
+477	1
+478	1
+479	1
+480	1
+481	1
+482	1
+483	1
+484	1
+485	1
+487	1
+489	1
+490	1
+491	1
+492	1
+493	1
+494	1
+495	1
+496	1
+497	1
+498	1
diff --git a/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e
+++ b/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5
index 326493a1d8cb65c016f9b00f4a47e80cba58c2d2..8bfbbf1e69f57b941cd53fc5ce6c786930f726be 100644
--- a/sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ b/sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5
@@ -306,4 +306,4 @@
 495	1
 496	1
 497	1
-498	1
\ No newline at end of file
+498	1
diff --git a/sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9
index 0e40f3f1a49a98a060f0b6aa2359e192f229cdc9..c156bd5f5d8c6d2a83bc805c27a96c2675270cce 100644
--- a/sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9
+++ b/sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9
@@ -306,4 +306,4 @@
 495	val_495	1
 496	val_496	1
 497	val_497	1
-498	val_498	1
\ No newline at end of file
+498	val_498	1
diff --git a/sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5
index df07a9da29f01512947648bed0e2a4f71e8798dc..93e965c7714038926c1e1173addaeed83556567f 100644
--- a/sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ b/sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5
@@ -306,4 +306,4 @@
 495	1
 496	1
 497	1
-498	3
\ No newline at end of file
+498	3
diff --git a/sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9
index cf0c065dc6532d0dc397574a1d6dcbc7fff4f723..742479ac713c5fadd563fc299ebe13ee56de6bce 100644
--- a/sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9
+++ b/sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9
@@ -306,4 +306,4 @@
 495	val_495	1
 496	val_496	1
 497	val_497	1
-498	val_498	3
\ No newline at end of file
+498	val_498	3
diff --git a/sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5
index 326493a1d8cb65c016f9b00f4a47e80cba58c2d2..8bfbbf1e69f57b941cd53fc5ce6c786930f726be 100644
--- a/sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ b/sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5
@@ -306,4 +306,4 @@
 495	1
 496	1
 497	1
-498	1
\ No newline at end of file
+498	1
diff --git a/sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9
index 4ef88d57aa72bdea6a818be11f40dbd6a421c9be..deaf15e46291049e879c779e977ef8495e74ed43 100644
--- a/sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9
+++ b/sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9
@@ -306,4 +306,4 @@
 95	val_95	1
 96	val_96	1
 97	val_97	1
-98	val_98	1
\ No newline at end of file
+98	val_98	1
diff --git a/sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5
index 326493a1d8cb65c016f9b00f4a47e80cba58c2d2..8bfbbf1e69f57b941cd53fc5ce6c786930f726be 100644
--- a/sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ b/sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5
@@ -306,4 +306,4 @@
 495	1
 496	1
 497	1
-498	1
\ No newline at end of file
+498	1
diff --git a/sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9
index 0e40f3f1a49a98a060f0b6aa2359e192f229cdc9..c156bd5f5d8c6d2a83bc805c27a96c2675270cce 100644
--- a/sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9
+++ b/sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9
@@ -306,4 +306,4 @@
 495	val_495	1
 496	val_496	1
 497	val_497	1
-498	val_498	1
\ No newline at end of file
+498	val_498	1
diff --git a/sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5
index 326493a1d8cb65c016f9b00f4a47e80cba58c2d2..8bfbbf1e69f57b941cd53fc5ce6c786930f726be 100644
--- a/sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ b/sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5
@@ -306,4 +306,4 @@
 495	1
 496	1
 497	1
-498	1
\ No newline at end of file
+498	1
diff --git a/sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9
index 4ef88d57aa72bdea6a818be11f40dbd6a421c9be..deaf15e46291049e879c779e977ef8495e74ed43 100644
--- a/sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9
+++ b/sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9
@@ -306,4 +306,4 @@
 95	val_95	1
 96	val_96	1
 97	val_97	1
-98	val_98	1
\ No newline at end of file
+98	val_98	1
diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817
+++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index ae9bbc3e2c2f78d59305aeac9978956a9afe8943..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,10 +0,0 @@
-0	1	00.0
-1	71	132828.0
-2	69	251142.0
-3	62	364008.0
-4	74	4105526.0
-5	6	5794.0
-6	5	6796.0
-7	6	71470.0
-8	8	81524.0
-9	7	92094.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby_map_ppr-6-c83b156356c269b6aef263640a4f6b7b
new file mode 100644
index 0000000000000000000000000000000000000000..1b9d97300aa0862c583f9a731a7a79098dde1d23
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-6-c83b156356c269b6aef263640a4f6b7b
@@ -0,0 +1,10 @@
+0	1	00.0
+1	71	132828.0
+2	69	251142.0
+3	62	364008.0
+4	74	4105526.0
+5	6	5794.0
+6	5	6796.0
+7	6	71470.0
+8	8	81524.0
+9	7	92094.0
diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index 64bb7c62c1885d3b88a4710c29adad2f02c36ca0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,10 +0,0 @@
-0	1	00.0	0	1
-1	71	132828.0	10044	71
-2	69	251142.0	15780	69
-3	62	364008.0	20119	62
-4	74	4105526.0	30965	74
-5	6	5794.0	278	6
-6	5	6796.0	331	5
-7	6	71470.0	447	6
-8	8	81524.0	595	8
-9	7	92094.0	577	7
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6
index 6b6a788e382dbde382eb9b5f5e3e5a9eb1e3ac9b..0b7e79a79bd11a50b9c5231d28643ff027b2f6ee 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6
+++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6
@@ -3,4 +3,4 @@
 4	1
 5	1
 8	1
-9	1
\ No newline at end of file
+9	1
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6
index c33eed60c8d5b383b91ae3f94ec6e1368455ba53..d63ed5dbe78bb5d1369e8ffc35d55e229405eec2 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6
+++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6
@@ -3,4 +3,4 @@
 8	1
 10	1
 16	1
-18	1
\ No newline at end of file
+18	1
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6
index 6b6a788e382dbde382eb9b5f5e3e5a9eb1e3ac9b..0b7e79a79bd11a50b9c5231d28643ff027b2f6ee 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6
+++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6
@@ -3,4 +3,4 @@
 4	1
 5	1
 8	1
-9	1
\ No newline at end of file
+9	1
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6
index c33eed60c8d5b383b91ae3f94ec6e1368455ba53..d63ed5dbe78bb5d1369e8ffc35d55e229405eec2 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6
+++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6
@@ -3,4 +3,4 @@
 8	1
 10	1
 16	1
-18	1
\ No newline at end of file
+18	1
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11
+++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd
deleted file mode 100644
index 922f4bfc83e448fc6bbc7ad46f209a1e2748595e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd
+++ /dev/null
@@ -1,10 +0,0 @@
-0	1	00.0	0	3
-1	71	116414.0	10044	115
-2	69	225571.0	15780	111
-3	62	332004.0	20119	99
-4	74	452763.0	30965	124
-5	6	5397.0	278	10
-6	5	6398.0	331	6
-7	6	7735.0	447	10
-8	8	8762.0	595	10
-9	7	91047.0	577	12
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e
deleted file mode 100644
index 106132fc993d30ca550bddb914750d819bde2b8a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e
+++ /dev/null
@@ -1,5 +0,0 @@
-5	6	5397.0	278	10
-6	5	6398.0	331	6
-7	6	7735.0	447	10
-8	8	8762.0	595	10
-9	7	91047.0	577	12
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231
deleted file mode 100644
index 65235356ea42516fe32b0416fab8af4cf2be3353..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231
+++ /dev/null
@@ -1,5 +0,0 @@
-0	1	00.0	0	3
-1	71	116414.0	10044	115
-2	69	225571.0	15780	111
-3	62	332004.0	20119	99
-4	74	452763.0	30965	124
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd
deleted file mode 100644
index 922f4bfc83e448fc6bbc7ad46f209a1e2748595e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd
+++ /dev/null
@@ -1,10 +0,0 @@
-0	1	00.0	0	3
-1	71	116414.0	10044	115
-2	69	225571.0	15780	111
-3	62	332004.0	20119	99
-4	74	452763.0	30965	124
-5	6	5397.0	278	10
-6	5	6398.0	331	6
-7	6	7735.0	447	10
-8	8	8762.0	595	10
-9	7	91047.0	577	12
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421
deleted file mode 100644
index 017878bc9bee575834e4ae9ffef79aa37008e037..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421
+++ /dev/null
@@ -1,10 +0,0 @@
-0	1	00.0	0	3
-1	4	1878.0	878	6
-1	5	1729.0	729	8
-1	6	11282.0	1282	12
-1	6	11494.0	1494	11
-1	7	11171.0	1171	11
-1	7	11516.0	1516	10
-1	8	11263.0	1263	10
-1	9	12294.0	2294	14
-1	9	12654.0	2654	16
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae
deleted file mode 100644
index f21a658e3c68f1f4225c16d328c7c12b566059e6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae
+++ /dev/null
@@ -1,32 +0,0 @@
-5	1	5102.0	102	2
-5	1	5116.0	116	2
-5	1	515.0	15	3
-5	1	553.0	53	1
-5	1	554.0	54	1
-5	1	557.0	57	1
-6	1	6134.0	134	2
-6	1	664.0	64	1
-6	1	665.0	65	1
-6	1	666.0	66	1
-6	1	669.0	69	1
-7	1	7144.0	144	2
-7	1	7152.0	152	2
-7	1	7210.0	210	3
-7	1	774.0	74	1
-7	1	777.0	77	1
-7	1	778.0	78	1
-8	1	8166.0	166	2
-8	1	8168.0	168	2
-8	1	88.0	8	1
-8	1	880.0	80	1
-8	1	882.0	82	1
-8	1	885.0	85	1
-8	1	886.0	86	1
-8	1	887.0	87	1
-9	1	9190.0	190	2
-9	1	9194.0	194	2
-9	1	9196.0	196	2
-9	1	9270.0	270	3
-9	1	99.0	9	1
-9	1	992.0	92	1
-9	1	996.0	96	1
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e
deleted file mode 100644
index 106132fc993d30ca550bddb914750d819bde2b8a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e
+++ /dev/null
@@ -1,5 +0,0 @@
-5	6	5397.0	278	10
-6	5	6398.0	331	6
-7	6	7735.0	447	10
-8	8	8762.0	595	10
-9	7	91047.0	577	12
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231
deleted file mode 100644
index 65235356ea42516fe32b0416fab8af4cf2be3353..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231
+++ /dev/null
@@ -1,5 +0,0 @@
-0	1	00.0	0	3
-1	71	116414.0	10044	115
-2	69	225571.0	15780	111
-3	62	332004.0	20119	99
-4	74	452763.0	30965	124
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0
+++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40
index dbc7aebdbbae531e1a99b9672a3254572ced261e..dcb604016a969288c75b5d3c6b41a52dc315f306 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40
+++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40
@@ -2,4 +2,4 @@
 6	5
 7	6
 8	8
-9	7
\ No newline at end of file
+9	7
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74
index ebf9d6978dec6001d9b7d7c6d0a86fea8aafa92b..cecfbbd281537e94bd7d37153a701f3c66b81509 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74
+++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74
@@ -2,4 +2,4 @@
 1	71	115
 2	69	111
 3	62	99
-4	74	124
\ No newline at end of file
+4	74	124
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003
+++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69
index a5ae9e2a62227f36bcbfa87dc52534208813b5bf..83d4ac248982363b8962e82c1c72d57927a67234 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69
+++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69
@@ -1,2 +1,2 @@
 100	2
-200	2
\ No newline at end of file
+200	2
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63
index 16c7a647a2344e1b1dbba05f9d2ce9ae5b2dbc06..ab65c1a003b5b4d2a1bc4f70cadbdf1de57871b5 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63
+++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63
@@ -1 +1 @@
-400	1
\ No newline at end of file
+400	1
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69
index 67f077e51a64701406ed13c008ad7bcb02490303..11c303a71007ccb650009e3a223dbf98bd67f041 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69
+++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69
@@ -1,2 +1,2 @@
 val_100	2
-val_200	2
\ No newline at end of file
+val_200	2
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63
index b06ad20135fbeb681d58912e2df65a40e1e6a6fb..1e4c20551b68b3ec4edf59f9c668af84e342dd11 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63
+++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63
@@ -1 +1 @@
-val_200	2
\ No newline at end of file
+val_200	2
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69
index a5ae9e2a62227f36bcbfa87dc52534208813b5bf..83d4ac248982363b8962e82c1c72d57927a67234 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69
+++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69
@@ -1,2 +1,2 @@
 100	2
-200	2
\ No newline at end of file
+200	2
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63
index 16c7a647a2344e1b1dbba05f9d2ce9ae5b2dbc06..ab65c1a003b5b4d2a1bc4f70cadbdf1de57871b5 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63
+++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63
@@ -1 +1 @@
-400	1
\ No newline at end of file
+400	1
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69
index 67f077e51a64701406ed13c008ad7bcb02490303..11c303a71007ccb650009e3a223dbf98bd67f041 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69
+++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69
@@ -1,2 +1,2 @@
 val_100	2
-val_200	2
\ No newline at end of file
+val_200	2
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63
index b06ad20135fbeb681d58912e2df65a40e1e6a6fb..1e4c20551b68b3ec4edf59f9c668af84e342dd11 100644
--- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63
+++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63
@@ -1 +1 @@
-val_200	2
\ No newline at end of file
+val_200	2
diff --git a/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb b/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb
index efe5cc7795c658bc96643e88fbd52954a4cb66aa..26dbfc34ad4fb98566296115decc92538b26113d 100644
--- a/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb
+++ b/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb
@@ -1 +1 @@
--30.33
\ No newline at end of file
+-30.33
diff --git a/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 b/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03
index efe5cc7795c658bc96643e88fbd52954a4cb66aa..26dbfc34ad4fb98566296115decc92538b26113d 100644
--- a/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03
+++ b/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03
@@ -1 +1 @@
--30.33
\ No newline at end of file
+-30.33
diff --git a/sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 b/sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 b/sql/hive/src/test/resources/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7
deleted file mode 100644
index f2a91fe3bfab0de05fa132871e8b65c3a9650640..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7
+++ /dev/null
@@ -1,13 +0,0 @@
-0	val_0	1
-2	val_2	1
-4	val_4	1
-5	val_5	1
-8	val_8	1
-9	val_9	1
-10	val_10	1
-11	val_11	1
-12	val_12	1
-15	val_15	1
-17	val_17	1
-18	val_18	1
-19	val_19	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f b/sql/hive/src/test/resources/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f
deleted file mode 100644
index c8e666cb01e8e741208686e4a1a43fbe01ab25b2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f
+++ /dev/null
@@ -1,14 +0,0 @@
-0	3
-5	3
-12	2
-15	2
-18	2
-2	1
-4	1
-8	1
-9	1
-10	1
-11	1
-17	1
-19	1
-20	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb b/sql/hive/src/test/resources/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb
deleted file mode 100644
index 96a824a81c589796a61441557ea69c91eb087562..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb
+++ /dev/null
@@ -1,3 +0,0 @@
-19	val_19	19	val_19
-18	val_18	18	val_18
-17	val_17	17	val_17
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d b/sql/hive/src/test/resources/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d
deleted file mode 100644
index a79396dac079ab11a1bd8959de049c8ec3f13c9a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d
+++ /dev/null
@@ -1,13 +0,0 @@
-0	1
-2	1
-4	1
-5	1
-8	1
-9	1
-10	1
-11	1
-12	1
-15	1
-17	1
-18	1
-19	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 b/sql/hive/src/test/resources/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7
deleted file mode 100644
index f2a91fe3bfab0de05fa132871e8b65c3a9650640..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7
+++ /dev/null
@@ -1,13 +0,0 @@
-0	val_0	1
-2	val_2	1
-4	val_4	1
-5	val_5	1
-8	val_8	1
-9	val_9	1
-10	val_10	1
-11	val_11	1
-12	val_12	1
-15	val_15	1
-17	val_17	1
-18	val_18	1
-19	val_19	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d b/sql/hive/src/test/resources/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d
deleted file mode 100644
index a79396dac079ab11a1bd8959de049c8ec3f13c9a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d
+++ /dev/null
@@ -1,13 +0,0 @@
-0	1
-2	1
-4	1
-5	1
-8	1
-9	1
-10	1
-11	1
-12	1
-15	1
-17	1
-18	1
-19	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f
index ae9bbc3e2c2f78d59305aeac9978956a9afe8943..1b9d97300aa0862c583f9a731a7a79098dde1d23 100644
--- a/sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f
@@ -7,4 +7,4 @@
 6	5	6796.0
 7	6	71470.0
 8	8	81524.0
-9	7	92094.0
\ No newline at end of file
+9	7	92094.0
diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index 64bb7c62c1885d3b88a4710c29adad2f02c36ca0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,10 +0,0 @@
-0	1	00.0	0	1
-1	71	132828.0	10044	71
-2	69	251142.0	15780	69
-3	62	364008.0	20119	62
-4	74	4105526.0	30965	74
-5	6	5794.0	278	6
-6	5	6796.0	331	5
-7	6	71470.0	447	6
-8	8	81524.0	595	8
-9	7	92094.0	577	7
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index 4e31460a412ba0602bfdce3e4ff8e5f4edccad1e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,5 +0,0 @@
-1	1
-2	1
-3	1
-7	1
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 b/sql/hive/src/test/resources/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8
deleted file mode 100644
index 10f4a1f5ff34c19123002fd494f41374fc4ae23e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8
+++ /dev/null
@@ -1,6 +0,0 @@
-1	11	1
-2	12	1
-3	13	1
-7	17	1
-8	18	1
-8	28	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index 4e31460a412ba0602bfdce3e4ff8e5f4edccad1e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,5 +0,0 @@
-1	1
-2	1
-3	1
-7	1
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index 4e31460a412ba0602bfdce3e4ff8e5f4edccad1e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,5 +0,0 @@
-1	1
-2	1
-3	1
-7	1
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7
deleted file mode 100644
index c1cc4ee2047734e534c59f2f7810f7839b49c678..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7
+++ /dev/null
@@ -1,5 +0,0 @@
-1	1	1
-1	2	1
-1	3	1
-1	7	1
-1	8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca
deleted file mode 100644
index 97a3b8c2f5977cab83292242f5148aa71a3a9aaa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca
+++ /dev/null
@@ -1,6 +0,0 @@
-1	1	11	1
-2	1	12	1
-3	1	13	1
-7	1	17	1
-8	1	18	1
-8	1	28	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7
deleted file mode 100644
index f0192040e147bec2ee136fcf44dee5ccf7ee4952..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7
+++ /dev/null
@@ -1,5 +0,0 @@
-1	2	1
-2	3	1
-3	4	1
-7	8	1
-8	9	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index b6c2eb98e5e49972158e774b22c8a812ffbc8baa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,5 +0,0 @@
-2	1
-4	1
-6	1
-14	1
-16	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index 8e7ee8a2b47bbd539510e6780239c7e83907bdb9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,10 +0,0 @@
-1	1
-1	1
-2	1
-2	1
-3	1
-3	1
-7	1
-7	1
-8	2
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index 0b2b54cd94e4f9c627fdea0fc245158750c9fd0b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,10 +0,0 @@
-1	1
-2	1
-2	1
-3	1
-4	1
-6	1
-7	1
-8	2
-14	1
-16	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index 3d4708b7c9d649f0aaafa840ac15a6b50f0a7eaf..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,5 +0,0 @@
-1	2
-2	2
-3	2
-7	2
-8	4
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index 4e31460a412ba0602bfdce3e4ff8e5f4edccad1e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,5 +0,0 @@
-1	1
-2	1
-3	1
-7	1
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca
deleted file mode 100644
index 97a3b8c2f5977cab83292242f5148aa71a3a9aaa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca
+++ /dev/null
@@ -1,6 +0,0 @@
-1	1	11	1
-2	1	12	1
-3	1	13	1
-7	1	17	1
-8	1	18	1
-8	1	28	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 b/sql/hive/src/test/resources/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49
deleted file mode 100644
index 7ca6b0b28a960c010cbd36d9760a7d7f13fdada0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49
+++ /dev/null
@@ -1,6 +0,0 @@
-1	1	11	2	1
-2	1	12	2	1
-3	1	13	2	1
-7	1	17	2	1
-8	1	18	2	1
-8	1	28	2	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca
deleted file mode 100644
index 97a3b8c2f5977cab83292242f5148aa71a3a9aaa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca
+++ /dev/null
@@ -1,6 +0,0 @@
-1	1	11	1
-2	1	12	1
-3	1	13	1
-7	1	17	1
-8	1	18	1
-8	1	28	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca
deleted file mode 100644
index 58e16ef3c0ef39fd1b21bc345422a007c1b10487..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca
+++ /dev/null
@@ -1,6 +0,0 @@
-1	2	11	1
-2	2	12	1
-3	2	13	1
-7	2	17	1
-8	2	18	1
-8	2	28	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715
deleted file mode 100644
index 4e31460a412ba0602bfdce3e4ff8e5f4edccad1e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715
+++ /dev/null
@@ -1,5 +0,0 @@
-1	1
-2	1
-3	1
-7	1
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd b/sql/hive/src/test/resources/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd
deleted file mode 100644
index 10f4a1f5ff34c19123002fd494f41374fc4ae23e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd
+++ /dev/null
@@ -1,6 +0,0 @@
-1	11	1
-2	12	1
-3	13	1
-7	17	1
-8	18	1
-8	28	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715
deleted file mode 100644
index d15db8c5d079f3d93be587272125e81574d811b0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715
+++ /dev/null
@@ -1 +0,0 @@
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 b/sql/hive/src/test/resources/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7
deleted file mode 100644
index 3d1609d961673b80d37e0514ed423d49e3428977..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7
+++ /dev/null
@@ -1,2 +0,0 @@
-8	18	1
-8	28	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 b/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2
index c6bb9dbfd649719deb67b8dc5ecc2a719a3db73e..bfca78293c98837f351db7d2c6d8f0b6adc68710 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2
+++ b/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2
@@ -1,2 +1,2 @@
 0
-11
\ No newline at end of file
+11
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e
+++ b/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634
+++ b/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 b/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2
index c6bb9dbfd649719deb67b8dc5ecc2a719a3db73e..bfca78293c98837f351db7d2c6d8f0b6adc68710 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2
+++ b/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2
@@ -1,2 +1,2 @@
 0
-11
\ No newline at end of file
+11
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 b/sql/hive/src/test/resources/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1
deleted file mode 100644
index ded2854cdf564b50d08beab46156ca6b81c7371e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1
+++ /dev/null
@@ -1,6 +0,0 @@
-1	3	3	0.0
-1	1	1	2.0
-1	1	1	4.0
-1	3	3	5.0
-1	1	1	8.0
-1	1	1	9.0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 b/sql/hive/src/test/resources/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8
deleted file mode 100644
index 487b4c4a5cc6fbfb16959d12de003e83cf84406f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8
+++ /dev/null
@@ -1,6 +0,0 @@
-0	1	3	3	0.0
-2	1	1	1	2.0
-4	1	1	1	4.0
-5	1	3	3	5.0
-8	1	1	1	8.0
-9	1	1	1	9.0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 b/sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015
deleted file mode 100644
index 1e8b314962144c26d5e0e50fd29d2ca327864913..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015
+++ /dev/null
@@ -1 +0,0 @@
-6
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d
deleted file mode 100644
index 1e8b314962144c26d5e0e50fd29d2ca327864913..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d
+++ /dev/null
@@ -1 +0,0 @@
-6
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d
deleted file mode 100644
index 1e8b314962144c26d5e0e50fd29d2ca327864913..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d
+++ /dev/null
@@ -1 +0,0 @@
-6
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 b/sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688
deleted file mode 100644
index 6a5fe2835fc56273b04612616aa426536c79f2a2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688
+++ /dev/null
@@ -1 +0,0 @@
-6	10	10	28.0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 b/sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86
index c5b99ed941efc616d417c175980ad3486a6da756..e6a233467dcf5fa4236f82cbd3ad518414c871b6 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86
+++ b/sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86
@@ -3,4 +3,4 @@
 13	1
 17	1
 18	1
-28	1
\ No newline at end of file
+28	1
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e
+++ b/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634
+++ b/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 b/sql/hive/src/test/resources/golden/groupby_sort_2-5-c0f14def6a135cc50cba364e810ce28e
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83
rename to sql/hive/src/test/resources/golden/groupby_sort_2-5-c0f14def6a135cc50cba364e810ce28e
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926
index 10f4a1f5ff34c19123002fd494f41374fc4ae23e..0f333f42821a05ce2401215c21df40189b1cafb8 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926
+++ b/sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926
@@ -3,4 +3,4 @@
 3	13	1
 7	17	1
 8	18	1
-8	28	1
\ No newline at end of file
+8	28	1
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f b/sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f
index 4e31460a412ba0602bfdce3e4ff8e5f4edccad1e..e7273779ac1b8d892bfd88579bd247b822fdd0c5 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f
+++ b/sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f
@@ -2,4 +2,4 @@
 2	1
 3	1
 7	1
-8	2
\ No newline at end of file
+8	2
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e
+++ b/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634
+++ b/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce b/sql/hive/src/test/resources/golden/groupby_sort_3-5-c0f14def6a135cc50cba364e810ce28e
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce
rename to sql/hive/src/test/resources/golden/groupby_sort_3-5-c0f14def6a135cc50cba364e810ce28e
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd
index 4e31460a412ba0602bfdce3e4ff8e5f4edccad1e..e7273779ac1b8d892bfd88579bd247b822fdd0c5 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd
+++ b/sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd
@@ -2,4 +2,4 @@
 2	1
 3	1
 7	1
-8	2
\ No newline at end of file
+8	2
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 b/sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170
index 10f4a1f5ff34c19123002fd494f41374fc4ae23e..0f333f42821a05ce2401215c21df40189b1cafb8 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170
+++ b/sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170
@@ -3,4 +3,4 @@
 3	13	1
 7	17	1
 8	18	1
-8	28	1
\ No newline at end of file
+8	28	1
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e
+++ b/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634
+++ b/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c b/sql/hive/src/test/resources/golden/groupby_sort_4-5-c0f14def6a135cc50cba364e810ce28e
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c
rename to sql/hive/src/test/resources/golden/groupby_sort_4-5-c0f14def6a135cc50cba364e810ce28e
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926
index 10f4a1f5ff34c19123002fd494f41374fc4ae23e..0f333f42821a05ce2401215c21df40189b1cafb8 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926
+++ b/sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926
@@ -3,4 +3,4 @@
 3	13	1
 7	17	1
 8	18	1
-8	28	1
\ No newline at end of file
+8	28	1
diff --git a/sql/hive/src/test/resources/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 b/sql/hive/src/test/resources/golden/groupby_sort_5-13-c0f14def6a135cc50cba364e810ce28e
similarity index 100%
rename from sql/hive/src/test/resources/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41
rename to sql/hive/src/test/resources/golden/groupby_sort_5-13-c0f14def6a135cc50cba364e810ce28e
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926
index 10f4a1f5ff34c19123002fd494f41374fc4ae23e..0f333f42821a05ce2401215c21df40189b1cafb8 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926
+++ b/sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926
@@ -3,4 +3,4 @@
 3	13	1
 7	17	1
 8	18	1
-8	28	1
\ No newline at end of file
+8	28	1
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e
+++ b/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb b/sql/hive/src/test/resources/golden/groupby_sort_5-20-c0f14def6a135cc50cba364e810ce28e
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb
rename to sql/hive/src/test/resources/golden/groupby_sort_5-20-c0f14def6a135cc50cba364e810ce28e
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f b/sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f
index 4e31460a412ba0602bfdce3e4ff8e5f4edccad1e..e7273779ac1b8d892bfd88579bd247b822fdd0c5 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f
+++ b/sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f
@@ -2,4 +2,4 @@
 2	1
 3	1
 7	1
-8	2
\ No newline at end of file
+8	2
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634
+++ b/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 b/sql/hive/src/test/resources/golden/groupby_sort_5-5-c0f14def6a135cc50cba364e810ce28e
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3
rename to sql/hive/src/test/resources/golden/groupby_sort_5-5-c0f14def6a135cc50cba364e810ce28e
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd
index 4e31460a412ba0602bfdce3e4ff8e5f4edccad1e..e7273779ac1b8d892bfd88579bd247b822fdd0c5 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd
+++ b/sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd
@@ -2,4 +2,4 @@
 2	1
 3	1
 7	1
-8	2
\ No newline at end of file
+8	2
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e
+++ b/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634
+++ b/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 b/sql/hive/src/test/resources/golden/groupby_sort_6-9-591e03d1cfc10821a601498df1ed6675
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1
rename to sql/hive/src/test/resources/golden/groupby_sort_6-9-591e03d1cfc10821a601498df1ed6675
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926
index 10f4a1f5ff34c19123002fd494f41374fc4ae23e..0f333f42821a05ce2401215c21df40189b1cafb8 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926
+++ b/sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926
@@ -3,4 +3,4 @@
 3	13	1
 7	17	1
 8	18	1
-8	28	1
\ No newline at end of file
+8	28	1
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e
+++ b/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634
+++ b/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 b/sql/hive/src/test/resources/golden/groupby_sort_7-5-43e94a517107a5bcf6fee78e6c88a1cc
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1
rename to sql/hive/src/test/resources/golden/groupby_sort_7-5-43e94a517107a5bcf6fee78e6c88a1cc
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d
index 7813681f5b41c028345ca62a2be376bae70b7f61..7ed6ff82de6bcc2a78243fc9c54d3ef5ac14da69 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d
+++ b/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d
@@ -1 +1 @@
-5
\ No newline at end of file
+5
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e
+++ b/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634
+++ b/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 b/sql/hive/src/test/resources/golden/groupby_sort_8-5-43e94a517107a5bcf6fee78e6c88a1cc
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793
rename to sql/hive/src/test/resources/golden/groupby_sort_8-5-43e94a517107a5bcf6fee78e6c88a1cc
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d
index 7813681f5b41c028345ca62a2be376bae70b7f61..7ed6ff82de6bcc2a78243fc9c54d3ef5ac14da69 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d
+++ b/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d
@@ -1 +1 @@
-5
\ No newline at end of file
+5
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a b/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a
+++ b/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e
+++ b/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634
+++ b/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 b/sql/hive/src/test/resources/golden/groupby_sort_9-5-43e94a517107a5bcf6fee78e6c88a1cc
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62
rename to sql/hive/src/test/resources/golden/groupby_sort_9-5-43e94a517107a5bcf6fee78e6c88a1cc
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c b/sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c
index 3d4708b7c9d649f0aaafa840ac15a6b50f0a7eaf..612dcbb640d465e40e93ef871d6c8daeb961b2d7 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c
+++ b/sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c
@@ -2,4 +2,4 @@
 2	2
 3	2
 7	2
-8	4
\ No newline at end of file
+8	4
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index 4e31460a412ba0602bfdce3e4ff8e5f4edccad1e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,5 +0,0 @@
-1	1
-2	1
-3	1
-7	1
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8
deleted file mode 100644
index 10f4a1f5ff34c19123002fd494f41374fc4ae23e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8
+++ /dev/null
@@ -1,6 +0,0 @@
-1	11	1
-2	12	1
-3	13	1
-7	17	1
-8	18	1
-8	28	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index 4e31460a412ba0602bfdce3e4ff8e5f4edccad1e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,5 +0,0 @@
-1	1
-2	1
-3	1
-7	1
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index 4e31460a412ba0602bfdce3e4ff8e5f4edccad1e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,5 +0,0 @@
-1	1
-2	1
-3	1
-7	1
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7
deleted file mode 100644
index c1cc4ee2047734e534c59f2f7810f7839b49c678..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7
+++ /dev/null
@@ -1,5 +0,0 @@
-1	1	1
-1	2	1
-1	3	1
-1	7	1
-1	8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca
deleted file mode 100644
index 97a3b8c2f5977cab83292242f5148aa71a3a9aaa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca
+++ /dev/null
@@ -1,6 +0,0 @@
-1	1	11	1
-2	1	12	1
-3	1	13	1
-7	1	17	1
-8	1	18	1
-8	1	28	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7
deleted file mode 100644
index f0192040e147bec2ee136fcf44dee5ccf7ee4952..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7
+++ /dev/null
@@ -1,5 +0,0 @@
-1	2	1
-2	3	1
-3	4	1
-7	8	1
-8	9	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index b6c2eb98e5e49972158e774b22c8a812ffbc8baa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,5 +0,0 @@
-2	1
-4	1
-6	1
-14	1
-16	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index 8e7ee8a2b47bbd539510e6780239c7e83907bdb9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,10 +0,0 @@
-1	1
-1	1
-2	1
-2	1
-3	1
-3	1
-7	1
-7	1
-8	2
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index 0b2b54cd94e4f9c627fdea0fc245158750c9fd0b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,10 +0,0 @@
-1	1
-2	1
-2	1
-3	1
-4	1
-6	1
-7	1
-8	2
-14	1
-16	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index 3d4708b7c9d649f0aaafa840ac15a6b50f0a7eaf..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,5 +0,0 @@
-1	2
-2	2
-3	2
-7	2
-8	4
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd
deleted file mode 100644
index 4e31460a412ba0602bfdce3e4ff8e5f4edccad1e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd
+++ /dev/null
@@ -1,5 +0,0 @@
-1	1
-2	1
-3	1
-7	1
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca
deleted file mode 100644
index 97a3b8c2f5977cab83292242f5148aa71a3a9aaa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca
+++ /dev/null
@@ -1,6 +0,0 @@
-1	1	11	1
-2	1	12	1
-3	1	13	1
-7	1	17	1
-8	1	18	1
-8	1	28	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49
deleted file mode 100644
index 7ca6b0b28a960c010cbd36d9760a7d7f13fdada0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49
+++ /dev/null
@@ -1,6 +0,0 @@
-1	1	11	2	1
-2	1	12	2	1
-3	1	13	2	1
-7	1	17	2	1
-8	1	18	2	1
-8	1	28	2	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca
deleted file mode 100644
index 97a3b8c2f5977cab83292242f5148aa71a3a9aaa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca
+++ /dev/null
@@ -1,6 +0,0 @@
-1	1	11	1
-2	1	12	1
-3	1	13	1
-7	1	17	1
-8	1	18	1
-8	1	28	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca
deleted file mode 100644
index 58e16ef3c0ef39fd1b21bc345422a007c1b10487..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca
+++ /dev/null
@@ -1,6 +0,0 @@
-1	2	11	1
-2	2	12	1
-3	2	13	1
-7	2	17	1
-8	2	18	1
-8	2	28	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715
deleted file mode 100644
index 4e31460a412ba0602bfdce3e4ff8e5f4edccad1e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715
+++ /dev/null
@@ -1,5 +0,0 @@
-1	1
-2	1
-3	1
-7	1
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd
deleted file mode 100644
index 10f4a1f5ff34c19123002fd494f41374fc4ae23e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd
+++ /dev/null
@@ -1,6 +0,0 @@
-1	11	1
-2	12	1
-3	13	1
-7	17	1
-8	18	1
-8	28	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715
deleted file mode 100644
index d15db8c5d079f3d93be587272125e81574d811b0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715
+++ /dev/null
@@ -1 +0,0 @@
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7
deleted file mode 100644
index 3d1609d961673b80d37e0514ed423d49e3428977..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7
+++ /dev/null
@@ -1,2 +0,0 @@
-8	18	1
-8	28	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e
+++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634
+++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a b/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a
+++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa b/sql/hive/src/test/resources/golden/groupby_sort_test_1-6-c0f14def6a135cc50cba364e810ce28e
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa
rename to sql/hive/src/test/resources/golden/groupby_sort_test_1-6-c0f14def6a135cc50cba364e810ce28e
diff --git a/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f b/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f
index a99cee758fe57580712c8a073c13af79f329889f..3cb614bdd84e85b9cfcf482973bfa09821b31b6d 100644
--- a/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f
+++ b/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f
@@ -1 +1 @@
-3556498
\ No newline at end of file
+3556498
diff --git a/sql/hive/src/test/resources/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf b/sql/hive/src/test/resources/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 b/sql/hive/src/test/resources/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 b/sql/hive/src/test/resources/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 b/sql/hive/src/test/resources/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 b/sql/hive/src/test/resources/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261
deleted file mode 100644
index d7c6f236687d8f3822b81c41bd4800dd5388f5c2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261
+++ /dev/null
@@ -1,5 +0,0 @@
-key                 	int                 	None                
-_bucketname         	string              	                    
-_offsets            	array<bigint>       	                    
-	 	 
-Detailed Table Information	Table(tableName:default__src_src_index_2__, dbName:default, owner:null, createTime:1389344545, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array<bigint>, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_2__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344545}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 b/sql/hive/src/test/resources/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 b/sql/hive/src/test/resources/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669
deleted file mode 100644
index 4c6ec0ba34bb8a93d697266e08d862d4c37e1a1c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669
+++ /dev/null
@@ -1,5 +0,0 @@
-key                 	int                 	None                
-_bucketname         	string              	                    
-_offsets            	array<bigint>       	                    
-	 	 
-Detailed Table Information	Table(tableName:src_idx_src_index_3, dbName:default, owner:null, createTime:1389344545, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array<bigint>, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/src_idx_src_index_3, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344545}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 b/sql/hive/src/test/resources/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 b/sql/hive/src/test/resources/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165
deleted file mode 100644
index 0c6af94247b851dc380c451380f814d07c3ea935..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165
+++ /dev/null
@@ -1,5 +0,0 @@
-key                 	int                 	None                
-_bucketname         	string              	                    
-_offsets            	array<bigint>       	                    
-	 	 
-Detailed Table Information	Table(tableName:default__src_src_index_4__, dbName:default, owner:null, createTime:1389344545, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array<bigint>, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_4__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=	, field.delim=
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 b/sql/hive/src/test/resources/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d b/sql/hive/src/test/resources/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d
deleted file mode 100644
index e4a5816e6f1cc28c8be87d8d35f9ce5d2ca53241..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d
+++ /dev/null
@@ -1,5 +0,0 @@
-key                 	int                 	None                
-_bucketname         	string              	                    
-_offsets            	array<bigint>       	                    
-	 	 
-Detailed Table Information	Table(tableName:default__src_src_index_5__, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array<bigint>, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_5__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{escape.delim=\, serialization.format=	, field.delim=
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad b/sql/hive/src/test/resources/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 b/sql/hive/src/test/resources/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4
deleted file mode 100644
index 3d8751ca47049b31188ae96a28a3fd834fad08c2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4
+++ /dev/null
@@ -1,5 +0,0 @@
-key                 	int                 	from deserializer   
-_bucketname         	string              	from deserializer   
-_offsets            	array<bigint>       	from deserializer   
-	 	 
-Detailed Table Information	Table(tableName:default__src_src_index_6__, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array<bigint>, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_6__, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344546}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f b/sql/hive/src/test/resources/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 b/sql/hive/src/test/resources/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 b/sql/hive/src/test/resources/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871
deleted file mode 100644
index 0e4852b3190526ec28cb9156d5ed7e332bd609c6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871
+++ /dev/null
@@ -1,5 +0,0 @@
-key                 	int                 	from deserializer   
-_bucketname         	string              	from deserializer   
-_offsets            	array<bigint>       	from deserializer   
-	 	 
-Detailed Table Information	Table(tableName:src_idx_src_index_7, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array<bigint>, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/src_idx_src_index_7, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344546}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e b/sql/hive/src/test/resources/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 b/sql/hive/src/test/resources/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31
deleted file mode 100644
index 41a5492fc5331e642108323b3c3e5dfa39aa51c9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31
+++ /dev/null
@@ -1,5 +0,0 @@
-key                 	int                 	None                
-_bucketname         	string              	                    
-_offsets            	array<bigint>       	                    
-	 	 
-Detailed Table Information	Table(tableName:default__src_src_index_8__, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array<bigint>, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344546}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 b/sql/hive/src/test/resources/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf b/sql/hive/src/test/resources/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf
deleted file mode 100644
index 8212bf8b7d2fbcd6caba15698949db830fb63b3c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf
+++ /dev/null
@@ -1,5 +0,0 @@
-key                 	int                 	None                
-_bucketname         	string              	                    
-_offsets            	array<bigint>       	                    
-	 	 
-Detailed Table Information	Table(tableName:default__src_src_index_9__, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array<bigint>, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_9__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{prop2=val2, prop1=val1, transient_lastDdlTime=1389344546}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca b/sql/hive/src/test/resources/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 b/sql/hive/src/test/resources/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 b/sql/hive/src/test/resources/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe b/sql/hive/src/test/resources/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f b/sql/hive/src/test/resources/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 b/sql/hive/src/test/resources/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 b/sql/hive/src/test/resources/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 b/sql/hive/src/test/resources/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 b/sql/hive/src/test/resources/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 b/sql/hive/src/test/resources/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 b/sql/hive/src/test/resources/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 b/sql/hive/src/test/resources/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 b/sql/hive/src/test/resources/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-37-9334418431eca405f13206bd8db42a1b b/sql/hive/src/test/resources/golden/index_creation-37-9334418431eca405f13206bd8db42a1b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 b/sql/hive/src/test/resources/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d b/sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d
deleted file mode 100644
index e8310385c56dc4bbe379f43400f3181f6a59f260..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d
+++ /dev/null
@@ -1 +0,0 @@
-src
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 b/sql/hive/src/test/resources/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 b/sql/hive/src/test/resources/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 b/sql/hive/src/test/resources/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-7-9334418431eca405f13206bd8db42a1b b/sql/hive/src/test/resources/golden/index_creation-7-9334418431eca405f13206bd8db42a1b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 b/sql/hive/src/test/resources/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c b/sql/hive/src/test/resources/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 b/sql/hive/src/test/resources/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/index_serde-0-db64b724719d27c7f0db4f51f5c4edaa
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/index_serde-0-db64b724719d27c7f0db4f51f5c4edaa
diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 b/sql/hive/src/test/resources/golden/index_serde-1-6560d12b69d55e5297a145ebc4bb0cb3
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28
rename to sql/hive/src/test/resources/golden/index_serde-1-6560d12b69d55e5297a145ebc4bb0cb3
diff --git a/sql/hive/src/test/resources/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 b/sql/hive/src/test/resources/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63
deleted file mode 100644
index 48522980f81a8506f37baa5d62129cb1d569c32c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63
+++ /dev/null
@@ -1,3 +0,0 @@
-number              	int                 	from deserializer   
-first_name          	string              	from deserializer   
-last_name           	string              	from deserializer   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b b/sql/hive/src/test/resources/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b
deleted file mode 100644
index 4ed570f9070eb7981fbd1a5c23b65ee59a5e07db..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b
+++ /dev/null
@@ -1,5 +0,0 @@
-7	Sylvester	McCoy
-8	Paul	McGann
-9	Christopher	Eccleston
-10	David	Tennant
-11	Matt	Smith
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 b/sql/hive/src/test/resources/golden/index_serde-10-c85e061ea9c5b90ca69b7450faad14b6
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62
rename to sql/hive/src/test/resources/golden/index_serde-10-c85e061ea9c5b90ca69b7450faad14b6
diff --git a/sql/hive/src/test/resources/golden/index_serde-11-123301a057d4a46072d0431e00e20c4b b/sql/hive/src/test/resources/golden/index_serde-11-123301a057d4a46072d0431e00e20c4b
new file mode 100644
index 0000000000000000000000000000000000000000..63d56733b58b0f9b6a637e5694d573b1c669bba2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/index_serde-11-123301a057d4a46072d0431e00e20c4b
@@ -0,0 +1,5 @@
+7	Sylvester	McCoy
+8	Paul	McGann
+9	Christopher	Eccleston
+10	David	Tennant
+11	Matt	Smith
diff --git a/sql/hive/src/test/resources/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 b/sql/hive/src/test/resources/golden/index_serde-11-309e916d683a1a12ab62565697cb0046
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad b/sql/hive/src/test/resources/golden/index_serde-12-309e916d683a1a12ab62565697cb0046
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad
rename to sql/hive/src/test/resources/golden/index_serde-12-309e916d683a1a12ab62565697cb0046
diff --git a/sql/hive/src/test/resources/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 b/sql/hive/src/test/resources/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 b/sql/hive/src/test/resources/golden/index_serde-13-d590fd7cb9d433143de490d75686dd4
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7
rename to sql/hive/src/test/resources/golden/index_serde-13-d590fd7cb9d433143de490d75686dd4
diff --git a/sql/hive/src/test/resources/golden/index_serde-2-f92d6c66d21791c11d2a822df04c1b63 b/sql/hive/src/test/resources/golden/index_serde-2-f92d6c66d21791c11d2a822df04c1b63
new file mode 100644
index 0000000000000000000000000000000000000000..e716294e919d1169bc8e1c524262681d5608fbc8
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/index_serde-2-f92d6c66d21791c11d2a822df04c1b63
@@ -0,0 +1,3 @@
+number              	int                 	from deserializer   
+first_name          	string              	from deserializer   
+last_name           	string              	from deserializer   
diff --git a/sql/hive/src/test/resources/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c b/sql/hive/src/test/resources/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 b/sql/hive/src/test/resources/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 b/sql/hive/src/test/resources/golden/index_serde-3-ebab588c84a7a29f03b41dcd98132229
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644
rename to sql/hive/src/test/resources/golden/index_serde-3-ebab588c84a7a29f03b41dcd98132229
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a b/sql/hive/src/test/resources/golden/index_serde-4-afcf2a156ccd4f79a0489b4593908d79
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a
rename to sql/hive/src/test/resources/golden/index_serde-4-afcf2a156ccd4f79a0489b4593908d79
diff --git a/sql/hive/src/test/resources/golden/index_serde-4-d7547751c37375a9238043bbe250e716 b/sql/hive/src/test/resources/golden/index_serde-4-d7547751c37375a9238043bbe250e716
deleted file mode 100644
index d2ca633d0ae013205a49c038aa3c53ae867f6063..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_serde-4-d7547751c37375a9238043bbe250e716
+++ /dev/null
@@ -1,5 +0,0 @@
-number              	int                 	from deserializer   
-_bucketname         	string              	                    
-_offsets            	array<bigint>       	                    
-	 	 
-Detailed Table Information	Table(tableName:default__doctors_doctors_index__, dbName:default, owner:null, createTime:1389729651, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:number, type:int, comment:from deserializer), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array<bigint>, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/default__doctors_doctors_index__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:number, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389729651}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/index_serde-5-d7547751c37375a9238043bbe250e716 b/sql/hive/src/test/resources/golden/index_serde-5-d7547751c37375a9238043bbe250e716
new file mode 100644
index 0000000000000000000000000000000000000000..c344129fb8f69b68f9d918975cab4cd8370025db
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/index_serde-5-d7547751c37375a9238043bbe250e716
@@ -0,0 +1,5 @@
+number              	int                 	from deserializer   
+_bucketname         	string              	                    
+_offsets            	array<bigint>       	                    
+	 	 
+Detailed Table Information	Table(tableName:default__doctors_doctors_index__, dbName:default, owner:marmbrus, createTime:1414101838, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:number, type:int, comment:from deserializer), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array<bigint>, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1438070583820061187/default__doctors_doctors_index__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:number, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1414101838}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 b/sql/hive/src/test/resources/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae b/sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 b/sql/hive/src/test/resources/golden/index_serde-6-e6ff4b23b7f102e359afb4d53a1dedc3
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_syntax-1-4f3d3497418242124113538edab45df7
rename to sql/hive/src/test/resources/golden/index_serde-6-e6ff4b23b7f102e359afb4d53a1dedc3
diff --git a/sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 b/sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/index_serde-7-c9d7dcde469d3b9a66965a64dd15e4ae
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/index_serde-7-c9d7dcde469d3b9a66965a64dd15e4ae
diff --git a/sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 b/sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/index_serde-8-3b03210f94ec40db9ab02620645014d1
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/index_serde-8-3b03210f94ec40db9ab02620645014d1
diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/index_serde-9-35f48c7d6fa164bb84643657bc9280a8
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/index_serde-9-35f48c7d6fa164bb84643657bc9280a8
diff --git a/sql/hive/src/test/resources/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 b/sql/hive/src/test/resources/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e b/sql/hive/src/test/resources/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 b/sql/hive/src/test/resources/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd b/sql/hive/src/test/resources/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a
+++ b/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 b/sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095
index 3b7cf42f96358a0121683a1989c27b457a22ca2f..821c3c8c892521775a1230419b9c89aa5af25fe9 100644
--- a/sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095
+++ b/sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095
@@ -1025,4 +1025,4 @@
 498	val_498
 498	val_498
 498	val_498
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 b/sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3
index 010e999c36749528c2be354410151fbb73739b78..e8a910f80f457de83ae39cb4b2a11ff29828e190 100644
--- a/sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3
+++ b/sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3
@@ -1,3 +1,3 @@
-a                   	int                 	None                
+a                   	int                 	                    
 	 	 
-Detailed Table Information	Table(tableName:test, dbName:default, owner:marmbrus, createTime:1389729862, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389729862}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:test, dbName:default, owner:marmbrus, createTime:1413881850, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413881850}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5 b/sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5
index c5c8d29fdd13e6dee2330bc99daedf917717dbbb..7aae61e5eb82f6ed0a954ef086bae93dca6546e8 100644
--- a/sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5
+++ b/sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5
@@ -497,4 +497,4 @@
 403	val_403
 400	val_400
 200	val_200
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f b/sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f
index c5c8d29fdd13e6dee2330bc99daedf917717dbbb..7aae61e5eb82f6ed0a954ef086bae93dca6546e8 100644
--- a/sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f
+++ b/sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f
@@ -497,4 +497,4 @@
 403	val_403
 400	val_400
 200	val_200
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 b/sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0
index e00bf4ec2c7e51c8696c8d6672e9857207d77a76..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 100644
--- a/sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0
+++ b/sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0
@@ -1,17 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_DESCTABLE (TOK_TABTYPE TEST1))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Describe Table Operator:
-        Describe Table
-          table: TEST1
-
-  Stage: Stage-1
-    Fetch Operator
-      limit: -1
-
diff --git a/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 b/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145
index 743be67e8d1c30456fa75b04b8ce56214970303f..d3ffb995aff4be6917ca104b39f82634b1491675 100644
--- a/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145
+++ b/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145
@@ -1,2 +1,2 @@
-a                   	int                 	None                
-b                   	double              	None                
\ No newline at end of file
+a                   	int                 	                    
+b                   	double              	                    
diff --git a/sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 b/sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74
index 175d371fd09c91c9b8dadbb7a87df0cb3860ecb9..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 100644
--- a/sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74
+++ b/sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74
@@ -1,17 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_DESCTABLE (TOK_TABTYPE TEST10))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Describe Table Operator:
-        Describe Table
-          table: TEST10
-
-  Stage: Stage-1
-    Fetch Operator
-      limit: -1
-
diff --git a/sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 b/sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449
index 8dcdf43e31be3c6b8c9b2867e0a437a2f679ffd9..4cb356c2355738c2ec3a002212ce8cf6b079babe 100644
--- a/sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449
+++ b/sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449
@@ -1,10 +1,10 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-hr                  	string              	None                
\ No newline at end of file
+ds                  	string              	                    
+hr                  	string              	                    
diff --git a/sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f
index 37dd922fd07872de24040f2a331386af744c3b9a..46057aa0a8fca630cef9f7484d48206163cd6cf9 100644
--- a/sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -81,4 +81,4 @@
 28	val_28
 37	val_37
 90	val_90
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 b/sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5
index fdf93911ee84741e3ee502231b5adb5c3acc9a49..d8bd3b13b83f121825bc8abbc492a1cf46f03116 100644
--- a/sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5
+++ b/sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5
@@ -7,4 +7,4 @@
 66	val_66
 82	val_82
 86	val_86
-98	val_98
\ No newline at end of file
+98	val_98
diff --git a/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e
+++ b/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880
+++ b/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c b/sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c
index 4220cf5f303924e9d9de9cd74e35b4d039848494..a66a07386eef8ccfbd888607fb80db4b453e8626 100644
--- a/sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c
+++ b/sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c
@@ -308,4 +308,4 @@
 222	2008-04-08	12
 403	2008-04-08	12
 400	2008-04-08	12
-200	2008-04-08	12
\ No newline at end of file
+200	2008-04-08	12
diff --git a/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3
+++ b/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f
index 37dd922fd07872de24040f2a331386af744c3b9a..46057aa0a8fca630cef9f7484d48206163cd6cf9 100644
--- a/sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f
@@ -81,4 +81,4 @@
 28	val_28
 37	val_37
 90	val_90
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd b/sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd
index d6e0c29932b9b196666834d9b6af8971b80371a8..9ee31317478d5118a91e39f6663b289f27d93fab 100644
--- a/sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd
+++ b/sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd
@@ -102,4 +102,4 @@
 152	val_152
 194	val_194
 126	val_126
-169	val_169
\ No newline at end of file
+169	val_169
diff --git a/sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f
index 703a4eef24f3ff147568d63701f1ff873f5bad82..0190981db84ed467a736e28ab815ae43b1258667 100644
--- a/sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -81,4 +81,4 @@
 97	val_97
 97	val_97
 98	val_98
-98	val_98
\ No newline at end of file
+98	val_98
diff --git a/sql/hive/src/test/resources/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf b/sql/hive/src/test/resources/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 b/sql/hive/src/test/resources/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index 4335dce6a9929b3c2f95678176bd19625c236c5d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,5 +0,0 @@
-0	val_0
-0	val_0
-0	val_0
-10	val_10
-11	val_11
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 b/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919
index ded361eb294f00071063d0c5a784b23c99058c85..90f9bd0430a4cafabf687a63ca77f97f946c1b22 100644
--- a/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919
+++ b/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919
@@ -1,2 +1,2 @@
-key                 	int                 	None                
-value               	string              	None                
+key                 	int                 	                    
+value               	string              	                    
diff --git a/sql/hive/src/test/resources/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 b/sql/hive/src/test/resources/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 b/sql/hive/src/test/resources/golden/input17-1-be9cde5e769f171f60f61a7739de8f17
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input17-2-21166e268096f6ec67f4f57ec333e901 b/sql/hive/src/test/resources/golden/input17-2-21166e268096f6ec67f4f57ec333e901
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index 1c48b3680a3ac92747a0749d2b6d245753583033..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,11 +0,0 @@
-NULL	NULL
--1461153966	{"myint":49,"mystring":"343","underscore_int":7}
--1952710705	{"myint":25,"mystring":"125","underscore_int":5}
--734328905	{"myint":16,"mystring":"64","underscore_int":4}
--751827636	{"myint":4,"mystring":"8","underscore_int":2}
-1244525196	{"myint":36,"mystring":"216","underscore_int":6}
-1638581586	{"myint":64,"mystring":"512","underscore_int":8}
-1712634731	{"myint":0,"mystring":"0","underscore_int":0}
-336964422	{"myint":81,"mystring":"729","underscore_int":9}
-465985201	{"myint":1,"mystring":"1","underscore_int":1}
-477111225	{"myint":9,"mystring":"27","underscore_int":3}
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc b/sql/hive/src/test/resources/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a b/sql/hive/src/test/resources/golden/input19-1-f32df514de8156b5f5b435eea2c9be40
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a
rename to sql/hive/src/test/resources/golden/input19-1-f32df514de8156b5f5b435eea2c9be40
diff --git a/sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 b/sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730
index 795dbe3a976eb1ca0da95547a7e74daeb0c193be..db9438946fc84d5d39977bfd2041592bfb94c2cf 100644
--- a/sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730
+++ b/sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730
@@ -1 +1 @@
-127.0.0.1	NULL	frank	10/Oct/2000:13:55:36 -0700	GET /apache_pb.gif HTTP/1.0	200	2326
\ No newline at end of file
+127.0.0.1	NULL	frank	10/Oct/2000:13:55:36 -0700	GET /apache_pb.gif HTTP/1.0	200	2326
diff --git a/sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 b/sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5
index fdf93911ee84741e3ee502231b5adb5c3acc9a49..d8bd3b13b83f121825bc8abbc492a1cf46f03116 100644
--- a/sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5
+++ b/sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5
@@ -7,4 +7,4 @@
 66	val_66
 82	val_82
 86	val_86
-98	val_98
\ No newline at end of file
+98	val_98
diff --git a/sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 b/sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23
index 8a8f1a1b8bbe1c8951680e1aba4d2e7cc2d039c3..f2f1112224cd7b566f60c182e077929ebcff09a2 100644
--- a/sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23
+++ b/sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23
@@ -2,4 +2,4 @@
 37	val_37
 66	val_66
 86	val_86
-98	val_98
\ No newline at end of file
+98	val_98
diff --git a/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 b/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16
index 743be67e8d1c30456fa75b04b8ce56214970303f..d3ffb995aff4be6917ca104b39f82634b1491675 100644
--- a/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16
+++ b/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16
@@ -1,2 +1,2 @@
-a                   	int                 	None                
-b                   	double              	None                
\ No newline at end of file
+a                   	int                 	                    
+b                   	double              	                    
diff --git a/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd b/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd
index 743be67e8d1c30456fa75b04b8ce56214970303f..d3ffb995aff4be6917ca104b39f82634b1491675 100644
--- a/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd
+++ b/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd
@@ -1,2 +1,2 @@
-a                   	int                 	None                
-b                   	double              	None                
\ No newline at end of file
+a                   	int                 	                    
+b                   	double              	                    
diff --git a/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b b/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b
index ca0726f517eebceffefe1a3c5834dee1284fd698..77eaef91c9c3f967e6444f0854a48dfeda7005f7 100644
--- a/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b
+++ b/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b
@@ -1,3 +1,3 @@
-a                   	array<int>          	None                
-b                   	double              	None                
-c                   	map<double,int>     	None                
\ No newline at end of file
+a                   	array<int>          	                    
+b                   	double              	                    
+c                   	map<double,int>     	                    
diff --git a/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c
index 2c861553f9aa268d1046fdffe20bb3be8ee947bf..2dd749277aa486a0aa6f91eee5838d004dd8bc27 100644
--- a/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c
@@ -1,4 +1,4 @@
 src
 srcpart
 test2a
-test2b
\ No newline at end of file
+test2b
diff --git a/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c
index 3e40a0c866d4de9fb8daca00d5a0af4268904f28..d2cb69524ba343acdb9e4f48277d6b6f5299a25a 100644
--- a/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c
@@ -1,3 +1,3 @@
 src
 srcpart
-test2b
\ No newline at end of file
+test2b
diff --git a/sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 b/sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1
index 5337f342fedd83e469c0f59d507f5758ebcfdfbb..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 100644
--- a/sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1
+++ b/sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1
@@ -1,17 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  TOK_SHOWTABLES
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Show Table Operator:
-        Show Tables
-          database name: default
-
-  Stage: Stage-1
-    Fetch Operator
-      limit: -1
-
diff --git a/sql/hive/src/test/resources/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 b/sql/hive/src/test/resources/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 b/sql/hive/src/test/resources/golden/input21-1-c45ad493e95150b580be778da6065f36
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106
rename to sql/hive/src/test/resources/golden/input21-1-c45ad493e95150b580be778da6065f36
diff --git a/sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe b/sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe
index c6c298df200ca0968ffa8f2be9b9bfba95297779..8b39955512bc72d1d55b637b54f4cf192beb6174 100644
--- a/sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe
+++ b/sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe
@@ -7,4 +7,4 @@ NULL	1	same	5
 NULL	NULL	same	6
 1.0	NULL	same	7
 1.0	1	same	8
-1.0	1	same	9
\ No newline at end of file
+1.0	1	same	9
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 b/sql/hive/src/test/resources/golden/input22-1-b663ec84da3f9d9b9594ea2da81b1442
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2
rename to sql/hive/src/test/resources/golden/input22-1-b663ec84da3f9d9b9594ea2da81b1442
diff --git a/sql/hive/src/test/resources/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 b/sql/hive/src/test/resources/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79 b/sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79
index 336ba0545635d46b71b5b5210184433b96a82ee0..891dedb34c1dd71edf498cfd69dbf70344646f84 100644
--- a/sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79
+++ b/sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79
@@ -7,4 +7,4 @@
 103
 103
 104
-104
\ No newline at end of file
+104
diff --git a/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 b/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5
+++ b/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 b/sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3
index b8fd0ab6545d6951a969bcc63323ea37f9015596..833b21e4d6e4da7fa8585e0e2ae5d38a41b737ec 100644
--- a/sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3
+++ b/sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3
@@ -2,4 +2,4 @@
 0	val_0	2008-04-08	11
 0	val_0	2008-04-08	11
 2	val_2	2008-04-08	11
-4	val_4	2008-04-08	11
\ No newline at end of file
+4	val_4	2008-04-08	11
diff --git a/sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 b/sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785
index 899417ee77ad8a51695f51d99d51be8e9b199f82..badda49814562a7fac6093378a8b7e549a8b8bea 100644
--- a/sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785
+++ b/sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785
@@ -2,4 +2,4 @@
 86	val_86
 27	val_27
 165	val_165
-255	val_255
\ No newline at end of file
+255	val_255
diff --git a/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d b/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d
index 743be67e8d1c30456fa75b04b8ce56214970303f..d3ffb995aff4be6917ca104b39f82634b1491675 100644
--- a/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d
+++ b/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d
@@ -1,2 +1,2 @@
-a                   	int                 	None                
-b                   	double              	None                
\ No newline at end of file
+a                   	int                 	                    
+b                   	double              	                    
diff --git a/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 b/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475
index 594b29ca1410f884979c4a5134bf88cadee27d38..bd673a6c1f1d47bb17e7c643ddc29aebf9cbc9b1 100644
--- a/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475
+++ b/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475
@@ -1,4 +1,4 @@
-a                   	array<int>          	None                
-b                   	double              	None                
-c                   	map<double,int>     	None                
-x                   	double              	None                
\ No newline at end of file
+a                   	array<int>          	                    
+b                   	double              	                    
+c                   	map<double,int>     	                    
+x                   	double              	                    
diff --git a/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c
index ac382c7369264b78d15950ff8ec80fab69efa5ba..f5b9883df09c0b9dc090daf71e575fff00b1ed74 100644
--- a/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c
@@ -1,4 +1,4 @@
 src
 srcpart
 test3a
-test3c
\ No newline at end of file
+test3c
diff --git a/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 b/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95
index cb17be511e8753b412b1928e76e228c7a5bf4c70..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 100644
--- a/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95
+++ b/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95
@@ -1,14 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ALTERTABLE_REPLACECOLS TEST3c (TOK_TABCOLLIST (TOK_TABCOL R1 TOK_INT) (TOK_TABCOL R2 TOK_DOUBLE)))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Alter Table Operator:
-        Alter Table
-          type: replace columns
-          new columns: r1 int, r2 double
-          old name: TEST3c
-
diff --git a/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 b/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219
index b906fd3c2e775af13d9c976135b712e1b866977c..ea55abd792314e0789b86f0488eb8535ff1cc60d 100644
--- a/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219
+++ b/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219
@@ -1,4 +1,4 @@
-r1                  	int                 	None                
-r2                  	double              	None                
+r1                  	int                 	                    
+r2                  	double              	                    
 	 	 
-Detailed Table Information	Table(tableName:test3c, dbName:default, owner:marmbrus, createTime:1389730377, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:r1, type:int, comment:null), FieldSchema(name:r2, type:double, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/test3c, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1389730378, transient_lastDdlTime=1389730378}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:test3c, dbName:default, owner:marmbrus, createTime:1413882084, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:r1, type:int, comment:null), FieldSchema(name:r2, type:double, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test3c, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413882084, transient_lastDdlTime=1413882084, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 b/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994
index ca0726f517eebceffefe1a3c5834dee1284fd698..77eaef91c9c3f967e6444f0854a48dfeda7005f7 100644
--- a/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994
+++ b/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994
@@ -1,3 +1,3 @@
-a                   	array<int>          	None                
-b                   	double              	None                
-c                   	map<double,int>     	None                
\ No newline at end of file
+a                   	array<int>          	                    
+b                   	double              	                    
+c                   	map<double,int>     	                    
diff --git a/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c
index 6b1ce270d97e9d29451ff89df602f550d97c3f10..b584fd7c6fd365782ec0243932df3b78a9c6d489 100644
--- a/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c
@@ -1,4 +1,4 @@
 src
 srcpart
 test3a
-test3b
\ No newline at end of file
+test3b
diff --git a/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 b/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2
index 92c0ed68f8a7b2bd64a994fac6964810b13274bf..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 100644
--- a/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2
+++ b/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2
@@ -1,14 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ALTERTABLE_ADDCOLS TEST3b (TOK_TABCOLLIST (TOK_TABCOL X TOK_DOUBLE)))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Alter Table Operator:
-        Alter Table
-          type: add columns
-          new columns: x double
-          old name: TEST3b
-
diff --git a/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 b/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994
index 594b29ca1410f884979c4a5134bf88cadee27d38..bd673a6c1f1d47bb17e7c643ddc29aebf9cbc9b1 100644
--- a/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994
+++ b/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994
@@ -1,4 +1,4 @@
-a                   	array<int>          	None                
-b                   	double              	None                
-c                   	map<double,int>     	None                
-x                   	double              	None                
\ No newline at end of file
+a                   	array<int>          	                    
+b                   	double              	                    
+c                   	map<double,int>     	                    
+x                   	double              	                    
diff --git a/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be b/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be
index 09bbc2937772095617b7acbfac86ebce72775c9b..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 100644
--- a/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be
+++ b/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be
@@ -1,14 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ALTERTABLE_RENAME TEST3b TEST3c)
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Alter Table Operator:
-        Alter Table
-          type: rename
-          new name: TEST3c
-          old name: TEST3b
-
diff --git a/sql/hive/src/test/resources/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c b/sql/hive/src/test/resources/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input30-1-90c0d1a75de78c405413fd627caea4ab b/sql/hive/src/test/resources/golden/input30-1-90c0d1a75de78c405413fd627caea4ab
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79 b/sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 b/sql/hive/src/test/resources/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 b/sql/hive/src/test/resources/golden/input30-5-38734677c27b5f90a8df5da6e6351c76
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 b/sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f b/sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f
deleted file mode 100644
index 25bf17fc5aaabd17402e77a2b16f95fbea7310d2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f
+++ /dev/null
@@ -1 +0,0 @@
-18
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input31-2-705764f8f7cab9378964af30b83f7fe b/sql/hive/src/test/resources/golden/input31-2-705764f8f7cab9378964af30b83f7fe
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input31-3-50c905261882f7fd8539fdd91e68151f b/sql/hive/src/test/resources/golden/input31-3-50c905261882f7fd8539fdd91e68151f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input31-4-2f886fa357df9342733551fa1b53f913 b/sql/hive/src/test/resources/golden/input31-4-2f886fa357df9342733551fa1b53f913
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input32-3-b0070890240c15d647af59f41b77ba3d b/sql/hive/src/test/resources/golden/input32-3-b0070890240c15d647af59f41b77ba3d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 b/sql/hive/src/test/resources/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input32-5-8789d32fc5b784fe2d171566732c573e b/sql/hive/src/test/resources/golden/input32-5-8789d32fc5b784fe2d171566732c573e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input37-0-86e2e274650fb56651607ea10d356fc0 b/sql/hive/src/test/resources/golden/input37-0-86e2e274650fb56651607ea10d356fc0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 b/sql/hive/src/test/resources/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input39-0-7bd12162381231be9d578797818957a7 b/sql/hive/src/test/resources/golden/input39-0-7bd12162381231be9d578797818957a7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 b/sql/hive/src/test/resources/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 b/sql/hive/src/test/resources/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 b/sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528
deleted file mode 100644
index 25bf17fc5aaabd17402e77a2b16f95fbea7310d2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528
+++ /dev/null
@@ -1 +0,0 @@
-18
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 b/sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 b/sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d b/sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 b/sql/hive/src/test/resources/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e b/sql/hive/src/test/resources/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input39-4-53453776bf062f28d371fc7336b7eae2 b/sql/hive/src/test/resources/golden/input39-4-53453776bf062f28d371fc7336b7eae2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe b/sql/hive/src/test/resources/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 b/sql/hive/src/test/resources/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 b/sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528
deleted file mode 100644
index 25bf17fc5aaabd17402e77a2b16f95fbea7310d2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528
+++ /dev/null
@@ -1 +0,0 @@
-18
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 b/sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 b/sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 b/sql/hive/src/test/resources/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e b/sql/hive/src/test/resources/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 b/sql/hive/src/test/resources/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 b/sql/hive/src/test/resources/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 b/sql/hive/src/test/resources/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 b/sql/hive/src/test/resources/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d b/sql/hive/src/test/resources/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece b/sql/hive/src/test/resources/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d b/sql/hive/src/test/resources/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input3_limit-5-3664b564747487df13a5d109837219b5 b/sql/hive/src/test/resources/golden/input3_limit-5-3664b564747487df13a5d109837219b5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 b/sql/hive/src/test/resources/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67
deleted file mode 100644
index 6731b0cb0baaa4e3c31bffde5358414ce02d3a22..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67
+++ /dev/null
@@ -1,20 +0,0 @@
-0	val_0
-0	val_0
-0	val_0
-0	val_1
-0	val_1
-1	val_2
-10	val_10
-10	val_11
-100	val_100
-100	val_100
-100	val_101
-100	val_101
-101	val_102
-102	val_103
-103	val_103
-103	val_103
-104	val_104
-104	val_104
-104	val_105
-104	val_105
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 b/sql/hive/src/test/resources/golden/input4-1-7ce0bc5e5feeb09bf3fc139e102fb00e
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4
rename to sql/hive/src/test/resources/golden/input4-1-7ce0bc5e5feeb09bf3fc139e102fb00e
diff --git a/sql/hive/src/test/resources/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 b/sql/hive/src/test/resources/golden/input4-1-c139adc70f9942e527142e3be7fd2b87
deleted file mode 100644
index 65a457b52b0a6a92a52a868f3086d64e14270f28..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input4-1-c139adc70f9942e527142e3be7fd2b87
+++ /dev/null
@@ -1,27 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_LOAD '/Users/marmbrus/workspace/hive/data/files/kv1.txt' (TOK_TAB (TOK_TABNAME INPUT4)) LOCAL)
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 depends on stages: Stage-0
-  Stage-2 depends on stages: Stage-1
-
-STAGE PLANS:
-  Stage: Stage-0
-    Copy
-      source: file:/Users/marmbrus/workspace/hive/data/files/kv1.txt
-      destination: file:/tmp/hive-marmbrus/hive_2014-01-14_12-16-46_262_527870677085258278-1/-ext-10000
-
-  Stage: Stage-1
-    Move Operator
-      tables:
-          replace: false
-          table:
-              input format: org.apache.hadoop.mapred.TextInputFormat
-              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              name: default.input4
-
-  Stage: Stage-2
-    Stats-Aggr Operator
-
diff --git a/sql/hive/src/test/resources/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 b/sql/hive/src/test/resources/golden/input4-2-b663ec84da3f9d9b9594ea2da81b1442
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491
rename to sql/hive/src/test/resources/golden/input4-2-b663ec84da3f9d9b9594ea2da81b1442
diff --git a/sql/hive/src/test/resources/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 b/sql/hive/src/test/resources/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 b/sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4
index 218c223b050b75c922242806ded49260d4e3744f..103b537db90c6055780bf31faeaf2bdd47e9ef2a 100644
--- a/sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4
+++ b/sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4
@@ -497,4 +497,4 @@ val_169	169
 val_403	403
 val_400	400
 val_200	200
-val_97	97
\ No newline at end of file
+val_97	97
diff --git a/sql/hive/src/test/resources/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb b/sql/hive/src/test/resources/golden/input40-1-a8adb8ae1d13607851431a1baf7578ba
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb
rename to sql/hive/src/test/resources/golden/input40-1-a8adb8ae1d13607851431a1baf7578ba
diff --git a/sql/hive/src/test/resources/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 b/sql/hive/src/test/resources/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 b/sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93
index c5c8d29fdd13e6dee2330bc99daedf917717dbbb..7aae61e5eb82f6ed0a954ef086bae93dca6546e8 100644
--- a/sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93
+++ b/sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93
@@ -497,4 +497,4 @@
 403	val_403
 400	val_400
 200	val_200
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e b/sql/hive/src/test/resources/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 b/sql/hive/src/test/resources/golden/input40-4-fdeea6b676c670b17c8d91e24a97a127
similarity index 100%
rename from sql/hive/src/test/resources/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11
rename to sql/hive/src/test/resources/golden/input40-4-fdeea6b676c670b17c8d91e24a97a127
diff --git a/sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 b/sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361
index 6bc66cd8fe19b1015fb15cbeaa09675d4352d55a..e4b818f03539d7e80d94a6f9e14ee5687b13560b 100644
--- a/sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361
+++ b/sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361
@@ -497,4 +497,4 @@
 97	val_97	2009-08-01
 97	val_97	2009-08-01
 98	val_98	2009-08-01
-98	val_98	2009-08-01
\ No newline at end of file
+98	val_98	2009-08-01
diff --git a/sql/hive/src/test/resources/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 b/sql/hive/src/test/resources/golden/input40-6-6651f53efc5d03ed2d43b9d7aecc0002
similarity index 100%
rename from sql/hive/src/test/resources/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549
rename to sql/hive/src/test/resources/golden/input40-6-6651f53efc5d03ed2d43b9d7aecc0002
diff --git a/sql/hive/src/test/resources/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 b/sql/hive/src/test/resources/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 b/sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361
index ee3dddd8cff71c5349343afbc33f80a1f5a318ee..4467e7af00c0e653e518d5aac122b4cec78bb4e6 100644
--- a/sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361
+++ b/sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361
@@ -997,4 +997,4 @@
 97	val_98	2009-08-01
 98	val_98	2009-08-01
 98	val_98	2009-08-01
-99	val_100	2009-08-01
\ No newline at end of file
+99	val_100	2009-08-01
diff --git a/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923
+++ b/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee b/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee
index 579784a58a66c365b4a65ba3c47d340cbb6c599c..61191cde2953b3f4c6f188d25e4788c6659b5778 100644
--- a/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee
+++ b/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee
@@ -1,2 +1,2 @@
 0
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/input43-0-2baba8070f3585debc14b6bb3c83607a b/sql/hive/src/test/resources/golden/input43-0-2baba8070f3585debc14b6bb3c83607a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 b/sql/hive/src/test/resources/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f b/sql/hive/src/test/resources/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca b/sql/hive/src/test/resources/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input46-3-c185163787977498a4b84f39f983c431 b/sql/hive/src/test/resources/golden/input46-3-c185163787977498a4b84f39f983c431
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 b/sql/hive/src/test/resources/golden/input46-4-7f05cde078d90c25780a7d5491b20c27
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab b/sql/hive/src/test/resources/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb b/sql/hive/src/test/resources/golden/input4_cb_delim-1-353d2238b781a117888a67bb7b2b2537
similarity index 100%
rename from sql/hive/src/test/resources/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb
rename to sql/hive/src/test/resources/golden/input4_cb_delim-1-353d2238b781a117888a67bb7b2b2537
diff --git a/sql/hive/src/test/resources/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 b/sql/hive/src/test/resources/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 b/sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617
index 218c223b050b75c922242806ded49260d4e3744f..103b537db90c6055780bf31faeaf2bdd47e9ef2a 100644
--- a/sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617
+++ b/sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617
@@ -497,4 +497,4 @@ val_169	169
 val_403	403
 val_400	400
 val_200	200
-val_97	97
\ No newline at end of file
+val_97	97
diff --git a/sql/hive/src/test/resources/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d b/sql/hive/src/test/resources/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af b/sql/hive/src/test/resources/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af
deleted file mode 100644
index 217a1915f8826bc03b781c2c9a872348e628a87f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af
+++ /dev/null
@@ -1,10 +0,0 @@
-0	val_0
-0	val_0
-0	val_0
-2	val_2
-4	val_4
-5	val_5
-5	val_5
-5	val_5
-8	val_8
-9	val_9
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input5-0-659e06570690cceeb3f37e10e855d2ea b/sql/hive/src/test/resources/golden/input5-0-659e06570690cceeb3f37e10e855d2ea
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 b/sql/hive/src/test/resources/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad b/sql/hive/src/test/resources/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index f8aa003a65bb45a9decb25ae290a532bb7e5a0d7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,11 +0,0 @@
-NULL	NULL
-[0,0,0]	[{"myint":0,"mystring":"0","underscore_int":0}]
-[1,2,3]	[{"myint":1,"mystring":"1","underscore_int":1}]
-[2,4,6]	[{"myint":4,"mystring":"8","underscore_int":2}]
-[3,6,9]	[{"myint":9,"mystring":"27","underscore_int":3}]
-[4,8,12]	[{"myint":16,"mystring":"64","underscore_int":4}]
-[5,10,15]	[{"myint":25,"mystring":"125","underscore_int":5}]
-[6,12,18]	[{"myint":36,"mystring":"216","underscore_int":6}]
-[7,14,21]	[{"myint":49,"mystring":"343","underscore_int":7}]
-[8,16,24]	[{"myint":64,"mystring":"512","underscore_int":8}]
-[9,18,27]	[{"myint":81,"mystring":"729","underscore_int":9}]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f
index b4dc488518f24bfe28babac5e3db5cd454cdd460..f3f63f08fcf70f2d817a364630790153f1e883f8 100644
--- a/sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -7,4 +7,4 @@ NULL	val_265
 NULL	val_193
 NULL	
 NULL	
-NULL	
\ No newline at end of file
+NULL	
diff --git a/sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f
index 0b8a8960a992bb09ed38a83e91c8c878f946af56..65cada3d45b2bfd1273f504f357dac897dc8fcf5 100644
--- a/sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -22,4 +22,4 @@ NULL	146
 NULL	406
 NULL	NULL
 NULL	NULL
-NULL	NULL
\ No newline at end of file
+NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f
index c87107a2f11683e489e8ff577f09483ef1c74571..416fbfb9e5228d12016cf4657ba9e3e6a7e22381 100644
--- a/sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -22,4 +22,4 @@ NULL	NULL	NULL
 NULL	NULL	NULL
 NULL	NULL	NULL
 NULL	NULL	NULL
-NULL	NULL	NULL
\ No newline at end of file
+NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e b/sql/hive/src/test/resources/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 b/sql/hive/src/test/resources/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be b/sql/hive/src/test/resources/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 b/sql/hive/src/test/resources/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7
deleted file mode 100644
index e782acd4d1e7f9f36ee6ad3bddb255a4873379fd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7
+++ /dev/null
@@ -1,11 +0,0 @@
-[0,0,0]	["0","0","0"]	{"key_0":"value_0"}	1712634731	record_0
-[1,2,3]	["10","100","1000"]	{"key_1":"value_1"}	465985200	record_1
-[2,4,6]	["20","200","2000"]	{"key_2":"value_2"}	-751827638	record_2
-[3,6,9]	["30","300","3000"]	{"key_3":"value_3"}	477111222	record_3
-[4,8,12]	["40","400","4000"]	{"key_4":"value_4"}	-734328909	record_4
-[5,10,15]	["50","500","5000"]	{"key_5":"value_5"}	-1952710710	record_5
-[6,12,18]	["60","600","6000"]	{"key_6":"value_6"}	1244525190	record_6
-[7,14,21]	["70","700","7000"]	{"key_7":"value_7"}	-1461153973	record_7
-[8,16,24]	["80","800","8000"]	{"key_8":"value_8"}	1638581578	record_8
-[9,18,27]	["90","900","9000"]	{"key_9":"value_9"}	336964413	record_9
-NULL	NULL	NULL	0	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 b/sql/hive/src/test/resources/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3
deleted file mode 100644
index 6038b8aa32884fcadfa9ec4dd91e0cc3e51f54fb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3
+++ /dev/null
@@ -1,11 +0,0 @@
-0	0	NULL	1712634731	record_0
-1	10	NULL	465985200	record_1
-2	20	NULL	-751827638	record_2
-3	30	NULL	477111222	record_3
-4	40	NULL	-734328909	record_4
-5	50	NULL	-1952710710	record_5
-6	60	NULL	1244525190	record_6
-7	70	NULL	-1461153973	record_7
-8	80	NULL	1638581578	record_8
-9	90	NULL	336964413	record_9
-NULL	NULL	NULL	0	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d b/sql/hive/src/test/resources/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc b/sql/hive/src/test/resources/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 b/sql/hive/src/test/resources/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index e782acd4d1e7f9f36ee6ad3bddb255a4873379fd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,11 +0,0 @@
-[0,0,0]	["0","0","0"]	{"key_0":"value_0"}	1712634731	record_0
-[1,2,3]	["10","100","1000"]	{"key_1":"value_1"}	465985200	record_1
-[2,4,6]	["20","200","2000"]	{"key_2":"value_2"}	-751827638	record_2
-[3,6,9]	["30","300","3000"]	{"key_3":"value_3"}	477111222	record_3
-[4,8,12]	["40","400","4000"]	{"key_4":"value_4"}	-734328909	record_4
-[5,10,15]	["50","500","5000"]	{"key_5":"value_5"}	-1952710710	record_5
-[6,12,18]	["60","600","6000"]	{"key_6":"value_6"}	1244525190	record_6
-[7,14,21]	["70","700","7000"]	{"key_7":"value_7"}	-1461153973	record_7
-[8,16,24]	["80","800","8000"]	{"key_8":"value_8"}	1638581578	record_8
-[9,18,27]	["90","900","9000"]	{"key_9":"value_9"}	336964413	record_9
-NULL	NULL	NULL	0	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 b/sql/hive/src/test/resources/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31
deleted file mode 100644
index 6038b8aa32884fcadfa9ec4dd91e0cc3e51f54fb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31
+++ /dev/null
@@ -1,11 +0,0 @@
-0	0	NULL	1712634731	record_0
-1	10	NULL	465985200	record_1
-2	20	NULL	-751827638	record_2
-3	30	NULL	477111222	record_3
-4	40	NULL	-734328909	record_4
-5	50	NULL	-1952710710	record_5
-6	60	NULL	1244525190	record_6
-7	70	NULL	-1461153973	record_7
-8	80	NULL	1638581578	record_8
-9	90	NULL	336964413	record_9
-NULL	NULL	NULL	0	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d b/sql/hive/src/test/resources/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 b/sql/hive/src/test/resources/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 b/sql/hive/src/test/resources/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c b/sql/hive/src/test/resources/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c b/sql/hive/src/test/resources/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c
deleted file mode 100644
index 6af528eab23a8f57250394fdaf6c6e3c0d066684..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c
+++ /dev/null
@@ -1,11 +0,0 @@
-{"key_0":"value_0"}
-{"key_1":"value_1"}
-{"key_2":"value_2"}
-{"key_3":"value_3"}
-{"key_4":"value_4"}
-{"key_5":"value_5"}
-{"key_6":"value_6"}
-{"key_7":"value_7"}
-{"key_8":"value_8"}
-{"key_9":"value_9"}
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc b/sql/hive/src/test/resources/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 b/sql/hive/src/test/resources/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97
deleted file mode 100644
index e782acd4d1e7f9f36ee6ad3bddb255a4873379fd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97
+++ /dev/null
@@ -1,11 +0,0 @@
-[0,0,0]	["0","0","0"]	{"key_0":"value_0"}	1712634731	record_0
-[1,2,3]	["10","100","1000"]	{"key_1":"value_1"}	465985200	record_1
-[2,4,6]	["20","200","2000"]	{"key_2":"value_2"}	-751827638	record_2
-[3,6,9]	["30","300","3000"]	{"key_3":"value_3"}	477111222	record_3
-[4,8,12]	["40","400","4000"]	{"key_4":"value_4"}	-734328909	record_4
-[5,10,15]	["50","500","5000"]	{"key_5":"value_5"}	-1952710710	record_5
-[6,12,18]	["60","600","6000"]	{"key_6":"value_6"}	1244525190	record_6
-[7,14,21]	["70","700","7000"]	{"key_7":"value_7"}	-1461153973	record_7
-[8,16,24]	["80","800","8000"]	{"key_8":"value_8"}	1638581578	record_8
-[9,18,27]	["90","900","9000"]	{"key_9":"value_9"}	336964413	record_9
-NULL	NULL	NULL	0	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db b/sql/hive/src/test/resources/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db
deleted file mode 100644
index 6038b8aa32884fcadfa9ec4dd91e0cc3e51f54fb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db
+++ /dev/null
@@ -1,11 +0,0 @@
-0	0	NULL	1712634731	record_0
-1	10	NULL	465985200	record_1
-2	20	NULL	-751827638	record_2
-3	30	NULL	477111222	record_3
-4	40	NULL	-734328909	record_4
-5	50	NULL	-1952710710	record_5
-6	60	NULL	1244525190	record_6
-7	70	NULL	-1461153973	record_7
-8	80	NULL	1638581578	record_8
-9	90	NULL	336964413	record_9
-NULL	NULL	NULL	0	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a b/sql/hive/src/test/resources/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 b/sql/hive/src/test/resources/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 b/sql/hive/src/test/resources/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c b/sql/hive/src/test/resources/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c
deleted file mode 100644
index 1bb008b44d6ee573a0d154e9423bf5f9c032f7b2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c
+++ /dev/null
@@ -1,11 +0,0 @@
-[0,0,0]
-[1,2,3]
-[2,4,6]
-[3,6,9]
-[4,8,12]
-[5,10,15]
-[6,12,18]
-[7,14,21]
-[8,16,24]
-[9,18,27]
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a b/sql/hive/src/test/resources/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 b/sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034
index 17f24d2991d14f78c55477ccf0c6611ecdd29d1b..03887aed6585233a024895ae52256a989c1b58b8 100644
--- a/sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034
+++ b/sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034
@@ -17,4 +17,4 @@
 369	val_369
 66	val_66
 128	val_128
-213	val_213
\ No newline at end of file
+213	val_213
diff --git a/sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb b/sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb
index 499e6b8ab6b7ed68fd01e54f8fa172cce1cce25f..d7a8f25b41301ec8d6a1bd3d230e31a65586444e 100644
--- a/sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb
+++ b/sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb
@@ -997,4 +997,4 @@
 403	val_403	2008-04-08	12
 400	val_400	2008-04-08	12
 200	val_200	2008-04-08	12
-97	val_97	2008-04-08	12
\ No newline at end of file
+97	val_97	2008-04-08	12
diff --git a/sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f
index 3e5ae10e4670abbdda3d8977ff489c123d81ce22..7ae7ecbe28de6fba6a6d1d7bfe9aa1c0557ae3db 100644
--- a/sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -81,4 +81,4 @@
 28	val_28	12	2008-04-08
 37	val_37	12	2008-04-08
 90	val_90	12	2008-04-08
-97	val_97	12	2008-04-08
\ No newline at end of file
+97	val_97	12	2008-04-08
diff --git a/sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 b/sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967
index 82116102c1f544baf8f3566a270347b51f225206..89c49ce857f5a8cc4613df620efc4682501da4a7 100644
--- a/sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967
+++ b/sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967
@@ -1,12 +1,12 @@
-a                   	string              	None                
-b                   	string              	None                
-ds                  	string              	None                
-ts                  	string              	None                
+a                   	string              	                    
+b                   	string              	                    
+ds                  	string              	                    
+ts                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-ts                  	string              	None                
+ds                  	string              	                    
+ts                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[2008 04 08, 10:11:12=455], dbName:default, tableName:part_special, createTime:1388798899, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6540137288252557391/part_special/ds=2008 04 08/ts=10%3A11%3A12%3D455, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388798899})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[2008 04 08, 10:11:12=455], dbName:default, tableName:part_special, createTime:1413882241, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/part_special/ds=2008 04 08/ts=10%3A11%3A12%3D455, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882241, COLUMN_STATS_ACCURATE=true, totalSize=4, numRows=1, rawDataSize=3})	
diff --git a/sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 b/sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09
index c8d0d5593006981b52c15736ec2c0ed6d578234a..6de1c02821c77d7a60dd4dbb6b86942468bfddf7 100644
--- a/sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09
+++ b/sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09
@@ -1 +1 @@
-1	2	2008 04 08	10:11:12=455
\ No newline at end of file
+1	2	2008 04 08	10:11:12=455
diff --git a/sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 b/sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967
index 869eb58e70d824332bfae17185e8298075f996a4..f5c60fae1925e96b9d4666e7cfa21141e6a19d8c 100644
--- a/sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967
+++ b/sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967
@@ -1,12 +1,12 @@
-a                   	string              	None                
-b                   	string              	None                
-ds                  	string              	None                
-ts                  	string              	None                
+a                   	string              	                    
+b                   	string              	                    
+ds                  	string              	                    
+ts                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-ts                  	string              	None                
+ds                  	string              	                    
+ts                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[2008 04 08, 10:11:12=455], dbName:default, tableName:part_special, createTime:1388798920, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6540137288252557391/part_special/ds=2008 04 08/ts=10%3A11%3A12%3D455, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1388798920, numRows=1, totalSize=4, rawDataSize=3})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[2008 04 08, 10:11:12=455], dbName:default, tableName:part_special, createTime:1413882252, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/part_special/ds=2008 04 08/ts=10%3A11%3A12%3D455, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882252, COLUMN_STATS_ACCURATE=true, totalSize=4, numRows=1, rawDataSize=3})	
diff --git a/sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 b/sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09
index c8d0d5593006981b52c15736ec2c0ed6d578234a..6de1c02821c77d7a60dd4dbb6b86942468bfddf7 100644
--- a/sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09
+++ b/sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09
@@ -1 +1 @@
-1	2	2008 04 08	10:11:12=455
\ No newline at end of file
+1	2	2008 04 08	10:11:12=455
diff --git a/sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 b/sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665
index fd945cc15d9cadbe5bafc2612c823509932136d2..883de3e945c463ee3e1c029d15cad584c7e78f37 100644
--- a/sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665
+++ b/sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665
@@ -81,4 +81,4 @@
 97	val_97	12	2008-04-08
 97	val_97	12	2008-04-08
 98	val_98	12	2008-04-08
-98	val_98	12	2008-04-08
\ No newline at end of file
+98	val_98	12	2008-04-08
diff --git a/sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 b/sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861
index f23877361f33b2966b63f319cc81e8ae9f4ba80b..b23aa272636540a7ed0dbc5804540f3a9aec752b 100644
--- a/sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861
+++ b/sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861
@@ -81,4 +81,4 @@
 97	val_97	12	2008-04-09
 97	val_97	12	2008-04-09
 98	val_98	12	2008-04-09
-98	val_98	12	2008-04-09
\ No newline at end of file
+98	val_98	12	2008-04-09
diff --git a/sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 b/sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730
index f4026a591a958fbe6b680ee771bfa632a2fc0d72..31b575a403f819f03e77de92bbd824545101019f 100644
--- a/sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730
+++ b/sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730
@@ -497,4 +497,4 @@
 403	val_403	2008-04-08	11
 400	val_400	2008-04-08	11
 200	val_200	2008-04-08	11
-97	val_97	2008-04-08	11
\ No newline at end of file
+97	val_97	2008-04-08	11
diff --git a/sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 b/sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928
index 853c3bc8df7f068d587918507306dbac42ada8bb..dfabe7827837cbd4cf06afd9bf9bd800e24fe6a3 100644
--- a/sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928
+++ b/sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928
@@ -165,4 +165,4 @@
 98	val_98	2008-04-08	11
 98	val_98	2008-04-08	11
 98	val_98	2008-04-08	12
-98	val_98	2008-04-08	12
\ No newline at end of file
+98	val_98	2008-04-08	12
diff --git a/sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 b/sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0
index 540ca86636f3c126dd40f382b22ea032a2241b69..ed8993a9cbd0c599c4e7aa5874060b00a8d8dea8 100644
--- a/sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0
+++ b/sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0
@@ -333,4 +333,4 @@
 98	val_98	2008-04-08	12
 98	val_98	2008-04-08	12
 98	val_98	2008-04-08	12
-98	val_98	2008-04-08	12
\ No newline at end of file
+98	val_98	2008-04-08	12
diff --git a/sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 b/sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474
index a1728c82f0b356bc89ed4786e2eaca2335d8418b..5ee171a64f7ab1d945730b2714746bd4f201e6ad 100644
--- a/sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474
+++ b/sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474
@@ -7,4 +7,4 @@
 255	val_255	2008-04-08	11
 278	val_278	2008-04-08	11
 98	val_98	2008-04-08	11
-484	val_484	2008-04-08	11
\ No newline at end of file
+484	val_484	2008-04-08	11
diff --git a/sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643 b/sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643
index 438355d7b06f53f7f8ad32c7f810807261a6cbba..b7704cd0a2f0fc64e23082b384b86471b1f406df 100644
--- a/sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643
+++ b/sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643
@@ -997,4 +997,4 @@
 498	val_498	2008-04-08	11
 498	val_498	2008-04-08	12
 498	val_498	2008-04-08	12
-498	val_498	2008-04-08	12
\ No newline at end of file
+498	val_498	2008-04-08	12
diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 b/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4
+++ b/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e b/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e
+++ b/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c b/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c
+++ b/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 b/sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9
index c5c8d29fdd13e6dee2330bc99daedf917717dbbb..7aae61e5eb82f6ed0a954ef086bae93dca6546e8 100644
--- a/sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9
+++ b/sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9
@@ -497,4 +497,4 @@
 403	val_403
 400	val_400
 200	val_200
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 b/sql/hive/src/test/resources/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f b/sql/hive/src/test/resources/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 b/sql/hive/src/test/resources/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index c1a6abba038e79febbae31c623bcd82051b4d3de..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,11 +0,0 @@
-0	0	NULL
-2	1	NULL
-4	8	value_2
-6	27	NULL
-8	64	NULL
-10	125	NULL
-12	216	NULL
-14	343	NULL
-16	512	NULL
-18	729	NULL
-NULL	NULL	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 b/sql/hive/src/test/resources/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 b/sql/hive/src/test/resources/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd b/sql/hive/src/test/resources/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index 51645b2a07a39deec0bad8911ea61839d7750f7d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,10 +0,0 @@
-3	1	1
-3	1	1
-3	1	1
-3	1	1
-3	1	1
-3	1	1
-3	1	1
-3	1	1
-3	1	1
-3	1	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac b/sql/hive/src/test/resources/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 b/sql/hive/src/test/resources/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11
deleted file mode 100644
index 373a573714f4e7f1cbf3c580d3e9083213fc7950..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11
+++ /dev/null
@@ -1,11 +0,0 @@
-NULL	[0]
-NULL	[1]
-NULL	[4]
-NULL	[9]
-NULL	[16]
-NULL	[25]
-NULL	[36]
-NULL	[49]
-NULL	[64]
-value_9	[81]
-NULL	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 b/sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 b/sql/hive/src/test/resources/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 b/sql/hive/src/test/resources/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9
deleted file mode 100644
index 7490d2d44d71a9a2cc8bea442691aa907d8f740a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9
+++ /dev/null
@@ -1,10 +0,0 @@
-NULL	[0]
-NULL	[1]
-NULL	[4]
-NULL	[9]
-NULL	[16]
-NULL	[25]
-NULL	[36]
-NULL	[49]
-NULL	[64]
-value_9	[81]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 b/sql/hive/src/test/resources/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 b/sql/hive/src/test/resources/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9
deleted file mode 100644
index 7490d2d44d71a9a2cc8bea442691aa907d8f740a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9
+++ /dev/null
@@ -1,10 +0,0 @@
-NULL	[0]
-NULL	[1]
-NULL	[4]
-NULL	[9]
-NULL	[16]
-NULL	[25]
-NULL	[36]
-NULL	[49]
-NULL	[64]
-value_9	[81]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 b/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57
index 679d54cb5cb5e0f42acbc1b532527de1124b36ad..3577c8a4318695f8545e9a414fc817a07939262b 100644
--- a/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57
+++ b/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57
@@ -1,10 +1,10 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-country             	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+country             	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-country             	string              	None                
+ds                  	string              	                    
+country             	string              	                    
diff --git a/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c b/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c
index ded361eb294f00071063d0c5a784b23c99058c85..90f9bd0430a4cafabf687a63ca77f97f946c1b22 100644
--- a/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c
+++ b/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c
@@ -1,2 +1,2 @@
-key                 	int                 	None                
-value               	string              	None                
+key                 	int                 	                    
+value               	string              	                    
diff --git a/sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 b/sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6
index d5a489a9a5ed54be042c33702e79d9f7315be3c2..454e27ff0c28d3f8ccd0bb93d7de29ebfe4bc17f 100644
--- a/sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6
+++ b/sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6
@@ -1,15 +1,15 @@
-viewtime            	string              	None                
-userid              	int                 	None                
-page_url            	string              	None                
-referrer_url        	string              	None                
-friends             	array<bigint>       	None                
-properties          	map<string,string>  	None                
+viewtime            	string              	                    
+userid              	int                 	                    
+page_url            	string              	                    
+referrer_url        	string              	                    
+friends             	array<bigint>       	                    
+properties          	map<string,string>  	                    
 ip                  	string              	IP Address of the User
-ds                  	string              	None                
-country             	string              	None                
+ds                  	string              	                    
+country             	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-country             	string              	None                
\ No newline at end of file
+ds                  	string              	                    
+country             	string              	                    
diff --git a/sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 b/sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865
index e0bedb0512cfc2c39f3854e4a70b917157146345..0ea7cee2a9cf9a54b12104e9fb89be5de81801e8 100644
--- a/sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865
+++ b/sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865
@@ -1,17 +1,17 @@
-viewtime            	string              	None                
-userid              	int                 	None                
-page_url            	string              	None                
-referrer_url        	string              	None                
-friends             	array<bigint>       	None                
-properties          	map<string,string>  	None                
+viewtime            	string              	                    
+userid              	int                 	                    
+page_url            	string              	                    
+referrer_url        	string              	                    
+friends             	array<bigint>       	                    
+properties          	map<string,string>  	                    
 ip                  	string              	IP Address of the User
-ds                  	string              	None                
-country             	string              	None                
+ds                  	string              	                    
+country             	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-country             	string              	None                
+ds                  	string              	                    
+country             	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:inputddl4, dbName:default, owner:marmbrus, createTime:1389731336, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:viewtime, type:string, comment:null), FieldSchema(name:userid, type:int, comment:null), FieldSchema(name:page_url, type:string, comment:null), FieldSchema(name:referrer_url, type:string, comment:null), FieldSchema(name:friends, type:array<bigint>, comment:null), FieldSchema(name:properties, type:map<string,string>, comment:null), FieldSchema(name:ip, type:string, comment:IP Address of the User), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/inputddl4, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:32, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[userid], sortCols:[Order(col:viewtime, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], parameters:{transient_lastDdlTime=1389731336, comment=This is the page view table}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:inputddl4, dbName:default, owner:marmbrus, createTime:1413882343, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:viewtime, type:string, comment:null), FieldSchema(name:userid, type:int, comment:null), FieldSchema(name:page_url, type:string, comment:null), FieldSchema(name:referrer_url, type:string, comment:null), FieldSchema(name:friends, type:array<bigint>, comment:null), FieldSchema(name:properties, type:map<string,string>, comment:null), FieldSchema(name:ip, type:string, comment:IP Address of the User), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/inputddl4, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:32, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[userid], sortCols:[Order(col:viewtime, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], parameters:{transient_lastDdlTime=1413882343, comment=This is the page view table}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 b/sql/hive/src/test/resources/golden/inputddl6-1-572c1abb70f09726d1ba77bdc884597b
similarity index 100%
rename from sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11
rename to sql/hive/src/test/resources/golden/inputddl6-1-572c1abb70f09726d1ba77bdc884597b
diff --git a/sql/hive/src/test/resources/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf b/sql/hive/src/test/resources/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 b/sql/hive/src/test/resources/golden/inputddl6-2-3a4def4b370f75c5fcc1174626490363
similarity index 100%
rename from sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85
rename to sql/hive/src/test/resources/golden/inputddl6-2-3a4def4b370f75c5fcc1174626490363
diff --git a/sql/hive/src/test/resources/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c b/sql/hive/src/test/resources/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c b/sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c
index f1427896e83307f4d7047cfa6950ada4a405092e..a8a418d9736d9ed849ddc9b3ef3f3afc61dcab21 100644
--- a/sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c
+++ b/sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c
@@ -1,10 +1,10 @@
-key                 	string              	None                
-value               	string              	None                
-ds                  	string              	None                
+key                 	string              	                    
+value               	string              	                    
+ds                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
+ds                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:inputddl6, dbName:default, owner:marmbrus, createTime:1389731342, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/inputddl6, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{numPartitions=2, numFiles=2, transient_lastDdlTime=1389731342, numRows=0, totalSize=11624, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:inputddl6, dbName:default, owner:marmbrus, createTime:1413882344, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/inputddl6, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1413882344}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 b/sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03
index 822897217e86759d524394ccc7b3984c055ef4b3..05507162a92443d8d54294dffd551b2f93afa8b1 100644
--- a/sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03
+++ b/sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03
@@ -1,10 +1,10 @@
-key                 	string              	None                
-value               	string              	None                
-ds                  	string              	None                
+key                 	string              	                    
+value               	string              	                    
+ds                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
+ds                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[2008-04-08], dbName:default, tableName:inputddl6, createTime:1389731342, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/inputddl6/ds=2008-04-08, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1389731342, numRows=0, totalSize=5812, rawDataSize=0})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[2008-04-08], dbName:default, tableName:inputddl6, createTime:1413882344, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/inputddl6/ds=2008-04-08, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882344, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=0, rawDataSize=0})	
diff --git a/sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 b/sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94
index 94bcaaee2408c2e2fa08db34c0277566a0de6315..6e2459ea22ee15e9e678d5a3d7325d4d813cd94d 100644
--- a/sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94
+++ b/sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94
@@ -1,2 +1,2 @@
 ds=2008-04-08
-ds=2008-04-09
\ No newline at end of file
+ds=2008-04-09
diff --git a/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 b/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94
index b12a9f82cd90af0b9efafb6772b3528f898a74f8..017a142ab30b732ce610d257df20b3fb12d5a7f8 100644
--- a/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94
+++ b/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94
@@ -1 +1 @@
-ds=2008-04-09
\ No newline at end of file
+ds=2008-04-09
diff --git a/sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 b/sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5
index a6c282ab6f573e2f7d774612265a7737ce7c082c..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 100644
--- a/sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5
+++ b/sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5
@@ -1,19 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_DESCTABLE (TOK_TABTYPE INPUTDDL6 (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-09'))) EXTENDED)
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Describe Table Operator:
-        Describe Table
-          partition:
-            ds 2008-04-09
-          table: INPUTDDL6
-
-  Stage: Stage-1
-    Fetch Operator
-      limit: -1
-
diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 b/sql/hive/src/test/resources/golden/inputddl7-1-7195712efb4910294f63303ebce24453
similarity index 100%
rename from sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7
rename to sql/hive/src/test/resources/golden/inputddl7-1-7195712efb4910294f63303ebce24453
diff --git a/sql/hive/src/test/resources/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 b/sql/hive/src/test/resources/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d b/sql/hive/src/test/resources/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c b/sql/hive/src/test/resources/golden/inputddl7-10-7c9248b56948716913d332bd712d69bd
similarity index 100%
rename from sql/hive/src/test/resources/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c
rename to sql/hive/src/test/resources/golden/inputddl7-10-7c9248b56948716913d332bd712d69bd
diff --git a/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 b/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63
+++ b/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 b/sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277
index ef633a4aa09e05beaaa2218a6a81a21111d40b2f..edfcdbb1211bb27758d7b0654deba5d5280fdd33 100644
--- a/sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277
+++ b/sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277
@@ -1,3 +1,3 @@
-name                	string              	None                
+name                	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:t1, dbName:default, owner:marmbrus, createTime:1389731349, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numPartitions=0, numFiles=1, transient_lastDdlTime=1389731349, numRows=0, totalSize=5812, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:t1, dbName:default, owner:marmbrus, createTime:1413882345, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=1, transient_lastDdlTime=1413882345, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=0, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a b/sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a
index 86d5d8a125fbebe6ac666896ce81fcff7e02ec87..86c9b459e36c509a2929527c34d3f0a9e4d2b62a 100644
--- a/sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a
+++ b/sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a
@@ -1,3 +1,3 @@
-name                	string              	None                
+name                	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:t2, dbName:default, owner:marmbrus, createTime:1389731362, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/t2, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numPartitions=0, numFiles=1, transient_lastDdlTime=1389731362, numRows=0, totalSize=10508, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:t2, dbName:default, owner:marmbrus, createTime:1413882355, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/t2, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=1, transient_lastDdlTime=1413882355, COLUMN_STATS_ACCURATE=true, totalSize=10508, numRows=0, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 b/sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145
index b14d78536150c736543338c1e649c26151058134..04956c2c1c4476c87752916892b0bff7a88a1509 100644
--- a/sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145
+++ b/sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145
@@ -1,9 +1,9 @@
-name                	string              	None                
-ds                  	string              	None                
+name                	string              	                    
+ds                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
+ds                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[2008-04-09], dbName:default, tableName:t3, createTime:1389731375, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/t3/ds=2008-04-09, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1389731375, numRows=0, totalSize=5812, rawDataSize=0})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[2008-04-09], dbName:default, tableName:t3, createTime:1413882365, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/t3/ds=2008-04-09, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882365, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=0, rawDataSize=0})	
diff --git a/sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe b/sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe
index e476a7fa3959b0e3fa8abafd3e92208fbae05c94..76ecadd3851a60f9691bb5f97f583ad9b1f7add2 100644
--- a/sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe
+++ b/sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe
@@ -1,9 +1,9 @@
-name                	string              	None                
-ds                  	string              	None                
+name                	string              	                    
+ds                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
+ds                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[2008-04-09], dbName:default, tableName:t4, createTime:1389731388, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/t4/ds=2008-04-09, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1389731388, numRows=0, totalSize=10508, rawDataSize=0})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[2008-04-09], dbName:default, tableName:t4, createTime:1413882375, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/t4/ds=2008-04-09, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882375, COLUMN_STATS_ACCURATE=true, totalSize=10508, numRows=0, rawDataSize=0})	
diff --git a/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 b/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291
+++ b/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 b/sql/hive/src/test/resources/golden/inputddl7-4-68715ba2c11220be62394c86453e6d54
similarity index 100%
rename from sql/hive/src/test/resources/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97
rename to sql/hive/src/test/resources/golden/inputddl7-4-68715ba2c11220be62394c86453e6d54
diff --git a/sql/hive/src/test/resources/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 b/sql/hive/src/test/resources/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 b/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68
+++ b/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 b/sql/hive/src/test/resources/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 b/sql/hive/src/test/resources/golden/inputddl7-7-59dd2d2556769e19bdc0a444f40f8a71
similarity index 100%
rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02
rename to sql/hive/src/test/resources/golden/inputddl7-7-59dd2d2556769e19bdc0a444f40f8a71
diff --git a/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 b/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508
+++ b/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 b/sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995
index 66e6efceed3dce3251561e760fa75c8f00f751ec..5166f3678f405bfc6b1ee176646176337bfe3122 100644
--- a/sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995
+++ b/sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995
@@ -4,13 +4,13 @@ lint                	array<int>          	from deserializer
 lstring             	array<string>       	from deserializer   
 lintstring          	array<struct<myint:int,mystring:string,underscore_int:int>>	from deserializer   
 mstringstring       	map<string,string>  	from deserializer   
-ds                  	string              	None                
-country             	string              	None                
+ds                  	string              	                    
+country             	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-country             	string              	None                
+ds                  	string              	                    
+country             	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:inputddl8, dbName:default, owner:marmbrus, createTime:1389731407, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/inputddl8, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:32, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.Complex, serialization.format=com.facebook.thrift.protocol.TBinaryProtocol}), bucketCols:[aint], sortCols:[Order(col:lint, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], parameters:{transient_lastDdlTime=1389731407, comment=This is a thrift based table}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:inputddl8, dbName:default, owner:marmbrus, createTime:1413882387, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/inputddl8, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:32, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.Complex, serialization.format=com.facebook.thrift.protocol.TBinaryProtocol}), bucketCols:[aint], sortCols:[Order(col:lint, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], parameters:{transient_lastDdlTime=1413882387, comment=This is a thrift based table}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-86653b3af59df59f225ee00ff5fc119f
similarity index 100%
rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9
rename to sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-86653b3af59df59f225ee00ff5fc119f
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-b7aaedd7d624af4e48637ff1acabe485 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-b7aaedd7d624af4e48637ff1acabe485
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-dece2650bf0615e566cd6c84181ce026 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-dece2650bf0615e566cd6c84181ce026
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-e81d45a5bec5642ec4b762f1c1a482af
similarity index 100%
rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277
rename to sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-e81d45a5bec5642ec4b762f1c1a482af
diff --git a/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d b/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d
+++ b/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef b/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef
+++ b/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef b/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef
index e37d32abba426c06b752a5e53f48c595c84e9270..83b33d238dab9943201aaf267f701e8ea5fc9268 100644
--- a/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef
+++ b/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef
@@ -1 +1 @@
-1000
\ No newline at end of file
+1000
diff --git a/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef b/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef
index 37021f4a27201e7257a4ac7a992b0b4ebba61a01..3d86ec6498f3f2c00a73220120de77934b185d8d 100644
--- a/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef
+++ b/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef
@@ -1 +1 @@
-1500
\ No newline at end of file
+1500
diff --git a/sql/hive/src/test/resources/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 b/sql/hive/src/test/resources/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b b/sql/hive/src/test/resources/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 b/sql/hive/src/test/resources/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb b/sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb
deleted file mode 100644
index 5e96d815b6b78bd2d9e2375685908fd12149e6d8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb
+++ /dev/null
@@ -1 +0,0 @@
--826625916
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 b/sql/hive/src/test/resources/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 b/sql/hive/src/test/resources/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 b/sql/hive/src/test/resources/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb b/sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb
deleted file mode 100644
index eb9dc5833c2f92c1d2b2287a24d4bfb0f48ba513..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb
+++ /dev/null
@@ -1 +0,0 @@
-10226524244
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 b/sql/hive/src/test/resources/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 b/sql/hive/src/test/resources/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb b/sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb
deleted file mode 100644
index 28ced898ab5379b55d2422a295a13b6cea99d195..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb
+++ /dev/null
@@ -1 +0,0 @@
-20453048488
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d b/sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d
deleted file mode 100644
index ae4ee13c08e7628701b925b8962108bd7643bf6e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d
+++ /dev/null
@@ -1 +0,0 @@
-200
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b b/sql/hive/src/test/resources/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 b/sql/hive/src/test/resources/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 b/sql/hive/src/test/resources/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 b/sql/hive/src/test/resources/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa b/sql/hive/src/test/resources/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 b/sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844
deleted file mode 100644
index 84d7f3929d86db0ab1a315aa532feeb93fc85b79..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844
+++ /dev/null
@@ -1 +0,0 @@
--27100860056
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 b/sql/hive/src/test/resources/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 b/sql/hive/src/test/resources/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 b/sql/hive/src/test/resources/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 b/sql/hive/src/test/resources/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 b/sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08
deleted file mode 100644
index ae4ee13c08e7628701b925b8962108bd7643bf6e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08
+++ /dev/null
@@ -1 +0,0 @@
-200
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 b/sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844
deleted file mode 100644
index 3395f3bcc7b51a70a9b1f7c051a5d10d06f4846d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844
+++ /dev/null
@@ -1 +0,0 @@
--24159954504
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 b/sql/hive/src/test/resources/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into2-8-452111285dda40205ee587de8e972896 b/sql/hive/src/test/resources/golden/insert_into2-8-452111285dda40205ee587de8e972896
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 b/sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844
deleted file mode 100644
index ee0a47c9f6e00d2353bde9629aa0e22508404590..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844
+++ /dev/null
@@ -1 +0,0 @@
--36239931656
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 b/sql/hive/src/test/resources/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 b/sql/hive/src/test/resources/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 b/sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7
deleted file mode 100644
index 5e96d815b6b78bd2d9e2375685908fd12149e6d8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7
+++ /dev/null
@@ -1 +0,0 @@
--826625916
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 b/sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87
deleted file mode 100644
index 2ed5a7da11dcd3695c5f57d0ba8069e9458f8a1c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87
+++ /dev/null
@@ -1 +0,0 @@
-9399898328
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 b/sql/hive/src/test/resources/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 b/sql/hive/src/test/resources/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into3-2-e51c25bae2408422a56826a263479468 b/sql/hive/src/test/resources/golden/insert_into3-2-e51c25bae2408422a56826a263479468
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d b/sql/hive/src/test/resources/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd b/sql/hive/src/test/resources/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 b/sql/hive/src/test/resources/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 b/sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7
deleted file mode 100644
index 00ffdd24b0cff682581dc3b73cb452213f7ff27b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7
+++ /dev/null
@@ -1 +0,0 @@
-7813690682
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 b/sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87
deleted file mode 100644
index eb9dc5833c2f92c1d2b2287a24d4bfb0f48ba513..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87
+++ /dev/null
@@ -1 +0,0 @@
-10226524244
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 b/sql/hive/src/test/resources/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 b/sql/hive/src/test/resources/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 b/sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 b/sql/hive/src/test/resources/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 b/sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159
deleted file mode 100644
index e2954bd63682e4f5b29956ab0d29b32e37bf8bdf..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159
+++ /dev/null
@@ -1 +0,0 @@
--1653251832
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa b/sql/hive/src/test/resources/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 b/sql/hive/src/test/resources/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b b/sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b
deleted file mode 100644
index e2954bd63682e4f5b29956ab0d29b32e37bf8bdf..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b
+++ /dev/null
@@ -1 +0,0 @@
--1653251832
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 b/sql/hive/src/test/resources/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 b/sql/hive/src/test/resources/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 b/sql/hive/src/test/resources/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b b/sql/hive/src/test/resources/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a b/sql/hive/src/test/resources/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 b/sql/hive/src/test/resources/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 b/sql/hive/src/test/resources/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 b/sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159
deleted file mode 100644
index 5e96d815b6b78bd2d9e2375685908fd12149e6d8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159
+++ /dev/null
@@ -1 +0,0 @@
--826625916
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 b/sql/hive/src/test/resources/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 b/sql/hive/src/test/resources/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 b/sql/hive/src/test/resources/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c b/sql/hive/src/test/resources/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 b/sql/hive/src/test/resources/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d b/sql/hive/src/test/resources/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 b/sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7
deleted file mode 100644
index 3b6ef434b37e71f17bc3a5c89ec89f4e3b8f69e0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7
+++ /dev/null
@@ -1 +0,0 @@
--18626052920
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 b/sql/hive/src/test/resources/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 b/sql/hive/src/test/resources/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 b/sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7
deleted file mode 100644
index 0744f3dae0e0ab2b14637aa50ecc080a905772aa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7
+++ /dev/null
@@ -1 +0,0 @@
--37252105840
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 b/sql/hive/src/test/resources/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c b/sql/hive/src/test/resources/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 b/sql/hive/src/test/resources/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 b/sql/hive/src/test/resources/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 b/sql/hive/src/test/resources/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 b/sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533
deleted file mode 100644
index c0066b75af40eee3ebe636c356029bf5586bb788..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533
+++ /dev/null
@@ -1 +0,0 @@
-481928560
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f b/sql/hive/src/test/resources/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 b/sql/hive/src/test/resources/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 b/sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533
deleted file mode 100644
index 0b4a44e064f855365d0abb080da725f615f00996..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533
+++ /dev/null
@@ -1 +0,0 @@
-963857120
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 b/sql/hive/src/test/resources/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba b/sql/hive/src/test/resources/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 b/sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902
deleted file mode 100644
index d541b5d2072333b461e9ba0d3681a588dbed5371..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902
+++ /dev/null
@@ -1 +0,0 @@
--35226404960
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 b/sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66
deleted file mode 100644
index c21f4017362c10777bae22d6659f60f8977ee1fa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66
+++ /dev/null
@@ -1,2 +0,0 @@
-ds=1
-ds=2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 b/sql/hive/src/test/resources/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d b/sql/hive/src/test/resources/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 b/sql/hive/src/test/resources/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d b/sql/hive/src/test/resources/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f b/sql/hive/src/test/resources/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 b/sql/hive/src/test/resources/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 b/sql/hive/src/test/resources/golden/insert_into6-6-f6e7141a435922193937aa10085b0656
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 b/sql/hive/src/test/resources/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 b/sql/hive/src/test/resources/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab b/sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab
deleted file mode 100644
index d541b5d2072333b461e9ba0d3681a588dbed5371..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab
+++ /dev/null
@@ -1 +0,0 @@
--35226404960
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 b/sql/hive/src/test/resources/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a b/sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b b/sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 b/sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 b/sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87
index 5707ed08e7e5406a7505b26df5b80ad66114cfac..13e88f30fc08c5b24e1d6380d831f1da68cb74ec 100644
--- a/sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87
+++ b/sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87
@@ -97,4 +97,4 @@
 9	val_9	5	val_5
 9	val_9	5	val_5
 9	val_9	8	val_8
-9	val_9	9	val_9
\ No newline at end of file
+9	val_9	9	val_9
diff --git a/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a
+++ b/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095 b/sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095
index 59821aeea3008af83bf6baa196e28b04461efcce..c9c79b85dfe24d2c5c9e7b278875729d91d6daff 100644
--- a/sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095
+++ b/sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095
@@ -1025,4 +1025,4 @@
 200	val_200
 200	val_200
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d b/sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d
index 59821aeea3008af83bf6baa196e28b04461efcce..c9c79b85dfe24d2c5c9e7b278875729d91d6daff 100644
--- a/sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d
+++ b/sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d
@@ -1025,4 +1025,4 @@
 200	val_200
 200	val_200
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 b/sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232
index c888cfd6a047958518a37e7f457be0895d67a340..e9016953d4e484c5d487ea1e1e02f107104db308 100644
--- a/sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232
+++ b/sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232
@@ -145,4 +145,4 @@
 90	val_90
 90	val_90
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f b/sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f
index 1780076ae513fa80d0cdc0f9a5eec99ad80e6628..598608f124d3871939aef03369886f88bd4feb49 100644
--- a/sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f
+++ b/sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f
@@ -229,4 +229,4 @@
 37	val_37
 37	val_37
 37	val_37
-37	val_37
\ No newline at end of file
+37	val_37
diff --git a/sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 b/sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5
index a82a9a22c6b87d22de4fd7722e854e9013c20821..f9890a4c2d194331e9f294fe770a68cf2a111237 100644
--- a/sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5
+++ b/sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5
@@ -161,4 +161,4 @@
 90	val_90
 90	val_90
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e
+++ b/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880
+++ b/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3
+++ b/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 b/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6
+++ b/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 b/sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3
index 941e6bb7c887ecbcabffc573748bd86d2e8693a7..bd6afa8dada84493f3eaff551d722f2c2d83f777 100644
--- a/sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3
+++ b/sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3
@@ -1749,4 +1749,4 @@
 403	val_403
 400	val_400
 200	val_200
-200	val_200
\ No newline at end of file
+200	val_200
diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f
+++ b/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3
+++ b/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 b/sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3
index 941e6bb7c887ecbcabffc573748bd86d2e8693a7..bd6afa8dada84493f3eaff551d722f2c2d83f777 100644
--- a/sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3
+++ b/sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3
@@ -1749,4 +1749,4 @@
 403	val_403
 400	val_400
 200	val_200
-200	val_200
\ No newline at end of file
+200	val_200
diff --git a/sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667 b/sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667
index c2e3ea8b0c8e2a5198fe64e3322be72886970306..b212e93a0a8c2795b471511b8d5c5aeae0af43e6 100644
--- a/sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667
+++ b/sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667
@@ -1025,4 +1025,4 @@
 498	val_498	498	val_498
 498	val_498	498	val_498
 498	val_498	498	val_498
-498	val_498	498	val_498
\ No newline at end of file
+498	val_498	498	val_498
diff --git a/sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f
index 66fafbdf72b63bb93673817ae0e18ca07b2da193..3df4716f0b05f86e8801d5d86bf0d194c85857e6 100644
--- a/sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -1025,4 +1025,4 @@
 200	val_200	200	val_200
 200	val_200	200	val_200
 97	val_97	97	val_97
-97	val_97	97	val_97
\ No newline at end of file
+97	val_97	97	val_97
diff --git a/sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 b/sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5
index 10628f6c64bd469c583ea042931b054814a351ab..947ed199b0d4d3f2b4708b94484242ca1f203a0e 100644
--- a/sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5
+++ b/sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5
@@ -307,4 +307,4 @@ NULL	NULL	NULL	7
 495	1	NULL	NULL
 496	1	NULL	NULL
 497	1	NULL	NULL
-498	3	NULL	NULL
\ No newline at end of file
+498	3	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 b/sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696
index 2945c31ed4f0061ebb121b40ac98519cfa0524a7..12a176c7bc7cb1b40a42d3811bb7dbe931d83e83 100644
--- a/sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696
+++ b/sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696
@@ -469,4 +469,4 @@
 200	val_400
 200	val_400
 97	val_194
-97	val_194
\ No newline at end of file
+97	val_194
diff --git a/sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 b/sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049
index bd6e20b8de1e47d861918c5b26439ab380aa9f24..7568f8cde28f5cb201c4ced1182c54f886552bbb 100644
--- a/sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049
+++ b/sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049
@@ -545,4 +545,4 @@ NULL	NULL	NULL	NULL	498	val_498
 5	val_5	5	val_5	5	val_5
 5	val_5	5	val_5	5	val_5
 8	val_8	8	val_8	8	val_8
-9	val_9	9	val_9	9	val_9
\ No newline at end of file
+9	val_9	9	val_9	9	val_9
diff --git a/sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 b/sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9
index bd6e20b8de1e47d861918c5b26439ab380aa9f24..7568f8cde28f5cb201c4ced1182c54f886552bbb 100644
--- a/sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9
+++ b/sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9
@@ -545,4 +545,4 @@ NULL	NULL	NULL	NULL	498	val_498
 5	val_5	5	val_5	5	val_5
 5	val_5	5	val_5	5	val_5
 8	val_8	8	val_8	8	val_8
-9	val_9	9	val_9	9	val_9
\ No newline at end of file
+9	val_9	9	val_9	9	val_9
diff --git a/sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f b/sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f
index 9672e21fa0323ee3c9a35fe60b37c1abaa0ca092..80c230cf4a09dfa7582adcc24494ac639b270894 100644
--- a/sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f
+++ b/sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f
@@ -497,4 +497,4 @@ NULL	NULL	NULL	NULL	496	val_496
 NULL	NULL	NULL	NULL	497	val_497
 NULL	NULL	NULL	NULL	498	val_498
 NULL	NULL	NULL	NULL	498	val_498
-NULL	NULL	NULL	NULL	498	val_498
\ No newline at end of file
+NULL	NULL	NULL	NULL	498	val_498
diff --git a/sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 b/sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681
index 5707ed08e7e5406a7505b26df5b80ad66114cfac..13e88f30fc08c5b24e1d6380d831f1da68cb74ec 100644
--- a/sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681
+++ b/sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681
@@ -97,4 +97,4 @@
 9	val_9	5	val_5
 9	val_9	5	val_5
 9	val_9	8	val_8
-9	val_9	9	val_9
\ No newline at end of file
+9	val_9	9	val_9
diff --git a/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 b/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812
+++ b/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 b/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06
+++ b/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa
index 183353e5c705bb6f9f32bbb2ee3f69c39359536b..d3d377e8ae74d20ba3c403bffbac066fcb3a5d47 100644
--- a/sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -34,4 +34,4 @@
 406	val_406	val_406
 406	val_406	val_406
 406	val_406	val_406
-406	val_406	val_406
\ No newline at end of file
+406	val_406	val_406
diff --git a/sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa
index 4c88927e5657907921a5ce35966bd309a7b28c27..120056ea10c6029591a4be60fe32c14a8306bb15 100644
--- a/sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -104,4 +104,4 @@
 98	val_98	val_98
 98	val_98	val_98
 98	val_98	val_98
-98	val_98	val_98
\ No newline at end of file
+98	val_98	val_98
diff --git a/sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 b/sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9
index 2b75023ea4a9f29c34a634ab80e8b3d883f7ea52..2adcbc0f14b62557303ab0fede472ec8f61abfe9 100644
--- a/sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9
+++ b/sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9
@@ -38,4 +38,4 @@ NULL	val_484	val_484
 406	val_406	val_406
 406	val_406	val_406
 406	val_406	val_406
-406	val_406	val_406
\ No newline at end of file
+406	val_406	val_406
diff --git a/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751
+++ b/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa
index da3c427cab73f24822fb824a1264c8188dccedb3..c998494e1c524f299e7d2df56449291a91b10269 100644
--- a/sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -104,4 +104,4 @@
 98	val_98
 98	val_98
 98	val_98
-98	val_98
\ No newline at end of file
+98	val_98
diff --git a/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751
+++ b/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa
index f553ce0ca41f36818d5476cd7894f56b7b7c46b0..c8445b6e2c7846b4ae8aeeeb1152bef13580a0b5 100644
--- a/sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -12,4 +12,4 @@
 401	1	5
 406	1	4
 66	1	1
-98	1	2
\ No newline at end of file
+98	1	2
diff --git a/sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f
index 8886dc7e8f229e37481267d79ae8a82358453156..9c33812fa7ea4d94f474995a631fd2b87f1bd83a 100644
--- a/sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -2651,4 +2651,4 @@
 97	val_97
 97	val_97
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa
index 71094ee7360db3e0af9ce2a3483569385a8060e4..16b313fc58f233e2c4b366410d2e1866975b5835 100644
--- a/sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -12,4 +12,4 @@
 311	3
 369	3
 401	5
-406	4
\ No newline at end of file
+406	4
diff --git a/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751
+++ b/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa
index 5d28208ab255c4c0b2497085e090f598ffea4993..7c33b34887d6bcc4887a04c5f13195de562e5da5 100644
--- a/sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -12,4 +12,4 @@
 401	1
 406	1
 66	1
-98	1
\ No newline at end of file
+98	1
diff --git a/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751
+++ b/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa
index a6538b605a81775dbc398fb0f28506a52e294e03..a4dde6240cac787518032fb26f6b45159a3627f1 100644
--- a/sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -82,4 +82,4 @@
 98	val_98	val_98
 98	val_98	val_98
 98	val_98	val_98
-98	val_98	val_98
\ No newline at end of file
+98	val_98	val_98
diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a b/sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a
index a6538b605a81775dbc398fb0f28506a52e294e03..a4dde6240cac787518032fb26f6b45159a3627f1 100644
--- a/sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a
+++ b/sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a
@@ -82,4 +82,4 @@
 98	val_98	val_98
 98	val_98	val_98
 98	val_98	val_98
-98	val_98	val_98
\ No newline at end of file
+98	val_98	val_98
diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9
index a6538b605a81775dbc398fb0f28506a52e294e03..a4dde6240cac787518032fb26f6b45159a3627f1 100644
--- a/sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9
+++ b/sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9
@@ -82,4 +82,4 @@
 98	val_98	val_98
 98	val_98	val_98
 98	val_98	val_98
-98	val_98	val_98
\ No newline at end of file
+98	val_98	val_98
diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9
index b1251794645cea3624f7fdb9185c4ab6000d7047..13c35f8a6c6ba51c11438a1a667cd941b3e1e58b 100644
--- a/sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9
+++ b/sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9
@@ -94,4 +94,4 @@ NULL	val_484	val_484
 98	val_98	val_98
 98	val_98	val_98
 98	val_98	val_98
-98	val_98	val_98
\ No newline at end of file
+98	val_98	val_98
diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9
index a6538b605a81775dbc398fb0f28506a52e294e03..a4dde6240cac787518032fb26f6b45159a3627f1 100644
--- a/sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9
+++ b/sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9
@@ -82,4 +82,4 @@
 98	val_98	val_98
 98	val_98	val_98
 98	val_98	val_98
-98	val_98	val_98
\ No newline at end of file
+98	val_98	val_98
diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9
index a6538b605a81775dbc398fb0f28506a52e294e03..a4dde6240cac787518032fb26f6b45159a3627f1 100644
--- a/sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9
+++ b/sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9
@@ -82,4 +82,4 @@
 98	val_98	val_98
 98	val_98	val_98
 98	val_98	val_98
-98	val_98	val_98
\ No newline at end of file
+98	val_98	val_98
diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 b/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6
+++ b/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a b/sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a
index a6538b605a81775dbc398fb0f28506a52e294e03..a4dde6240cac787518032fb26f6b45159a3627f1 100644
--- a/sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a
+++ b/sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a
@@ -82,4 +82,4 @@
 98	val_98	val_98
 98	val_98	val_98
 98	val_98	val_98
-98	val_98	val_98
\ No newline at end of file
+98	val_98	val_98
diff --git a/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751
+++ b/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa
index a6538b605a81775dbc398fb0f28506a52e294e03..a4dde6240cac787518032fb26f6b45159a3627f1 100644
--- a/sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -82,4 +82,4 @@
 98	val_98	val_98
 98	val_98	val_98
 98	val_98	val_98
-98	val_98	val_98
\ No newline at end of file
+98	val_98	val_98
diff --git a/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751
+++ b/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa
index e18d1ff802c93b45a891980bd2d6ee9ee7aea2ec..6fbe456119b70b3dcc600a57fc816b7f658ab141 100644
--- a/sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -31,4 +31,4 @@
 406	val_406	val_406
 406	val_406	val_406
 406	val_406	val_406
-406	val_406	val_406
\ No newline at end of file
+406	val_406	val_406
diff --git a/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751
+++ b/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa
index efdd80c9f8b89a53af4bbf48f22573d1ceb511ee..2f62508e3342a7c2ca92ebfaf906eb942d21e353 100644
--- a/sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -10,4 +10,4 @@
 311	val_311	3
 369		3
 401	val_401	5
-406	val_406	4
\ No newline at end of file
+406	val_406	4
diff --git a/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 b/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06
+++ b/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa
index a250f202c4df0a231201c1c27f932383d176336c..4cb7ec20b1a33baf29065c22221e22ea25daac00 100644
--- a/sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -306,4 +306,4 @@
 495	1	1
 496	1	1
 497	1	1
-498	3	3
\ No newline at end of file
+498	3	3
diff --git a/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 b/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06
+++ b/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa
index 183353e5c705bb6f9f32bbb2ee3f69c39359536b..d3d377e8ae74d20ba3c403bffbac066fcb3a5d47 100644
--- a/sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -34,4 +34,4 @@
 406	val_406	val_406
 406	val_406	val_406
 406	val_406	val_406
-406	val_406	val_406
\ No newline at end of file
+406	val_406	val_406
diff --git a/sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9 b/sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9
index 5e020e3aff653e6a08ae456122a3aa35dfa0270e..3dcf0e17586a9daf46e05d354bc1e7a4bc3a1c55 100644
--- a/sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9
+++ b/sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9
@@ -1,2 +1,2 @@
 100	101	102	103	104	105	106	107	108	109	110	111
-100	101	102	103	104	105	106	107	108	109	110	111
\ No newline at end of file
+100	101	102	103	104	105	106	107	108	109	110	111
diff --git a/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef b/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef
index fce97c9211916b4eefd3fa28f7f296767df890cd..edbff1207ded97e8f7631312f7ac86aed0199e59 100644
--- a/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef
+++ b/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef
@@ -1 +1 @@
-val_111	105	2
\ No newline at end of file
+val_111	105	2
diff --git a/sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa
index 5b45955ceb27476329e919a60caf7dc22d098316..56d4dbe5b4d250362822e0ceeca62cf1fc04114b 100644
--- a/sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -563,4 +563,4 @@
 98	val_98	98	val_98
 98	val_98	98	val_98
 98	val_98	98	val_98
-98	val_98	98	val_98
\ No newline at end of file
+98	val_98	98	val_98
diff --git a/sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f
index e45f3234d5a887b13f365f9bbf4c741b90db83a1..dc3a273d7a367ee43fb7b541d1d0c9be3bdefa1f 100644
--- a/sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -8,4 +8,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac b/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac
+++ b/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa b/sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa
index 8543fe173f87fa702d13f0f775a6ef636f99a144..50237fbde9e34ed805940d78b8e8a9b5403a10d7 100644
--- a/sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa
+++ b/sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa
@@ -563,4 +563,4 @@
 400	val_400	NULL	NULL
 200	val_200	NULL	NULL
 97	val_97	97	val_97
-97	val_97	97	val_97
\ No newline at end of file
+97	val_97	97	val_97
diff --git a/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda b/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda
index 67d892c80f49383b62e4332582a9483d07de7831..1f3d8a7a1fc08495c982b22af6f815d5955a9928 100644
--- a/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda
+++ b/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda
@@ -1 +1 @@
-1028
\ No newline at end of file
+1028
diff --git a/sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 b/sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251
index 8543fe173f87fa702d13f0f775a6ef636f99a144..50237fbde9e34ed805940d78b8e8a9b5403a10d7 100644
--- a/sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251
+++ b/sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251
@@ -563,4 +563,4 @@
 400	val_400	NULL	NULL
 200	val_200	NULL	NULL
 97	val_97	97	val_97
-97	val_97	97	val_97
\ No newline at end of file
+97	val_97	97	val_97
diff --git a/sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 b/sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7
index 59821aeea3008af83bf6baa196e28b04461efcce..c9c79b85dfe24d2c5c9e7b278875729d91d6daff 100644
--- a/sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7
+++ b/sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7
@@ -1025,4 +1025,4 @@
 200	val_200
 200	val_200
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 b/sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049
index bd6e20b8de1e47d861918c5b26439ab380aa9f24..7568f8cde28f5cb201c4ced1182c54f886552bbb 100644
--- a/sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049
+++ b/sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049
@@ -545,4 +545,4 @@ NULL	NULL	NULL	NULL	498	val_498
 5	val_5	5	val_5	5	val_5
 5	val_5	5	val_5	5	val_5
 8	val_8	8	val_8	8	val_8
-9	val_9	9	val_9	9	val_9
\ No newline at end of file
+9	val_9	9	val_9	9	val_9
diff --git a/sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 b/sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9
index bd6e20b8de1e47d861918c5b26439ab380aa9f24..7568f8cde28f5cb201c4ced1182c54f886552bbb 100644
--- a/sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9
+++ b/sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9
@@ -545,4 +545,4 @@ NULL	NULL	NULL	NULL	498	val_498
 5	val_5	5	val_5	5	val_5
 5	val_5	5	val_5	5	val_5
 8	val_8	8	val_8	8	val_8
-9	val_9	9	val_9	9	val_9
\ No newline at end of file
+9	val_9	9	val_9	9	val_9
diff --git a/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 b/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917
+++ b/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 b/sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988
index 7265626e5dbdebaa87b76301b6987b82628e5e7b..c20eb16d47f913d4e2bda72da7542eaa61d44f93 100644
--- a/sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988
+++ b/sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988
@@ -1,3 +1,3 @@
 0	val_0	NULL	NULL
 0	val_0	NULL	NULL
-0	val_0	NULL	NULL
\ No newline at end of file
+0	val_0	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 b/sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988
index 7265626e5dbdebaa87b76301b6987b82628e5e7b..c20eb16d47f913d4e2bda72da7542eaa61d44f93 100644
--- a/sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988
+++ b/sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988
@@ -1,3 +1,3 @@
 0	val_0	NULL	NULL
 0	val_0	NULL	NULL
-0	val_0	NULL	NULL
\ No newline at end of file
+0	val_0	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f
index 06b2b4d7e6d4767f51ce7559c1a3b99cd420b5b8..b52cff5c472e4cce9ece568cdb1b7f57616f6e3b 100644
--- a/sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -6,4 +6,4 @@ NULL	NULL	24	val_24
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f
index 36ca4a0805f4bbf29e6a2d15514a38788ef7d549..fb58885263569410c324e5e215fabb03e254af71 100644
--- a/sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -11,4 +11,4 @@
 19	val_19	19	val_19
 NULL	NULL	20	val_20
 NULL	NULL	24	val_24
-NULL	NULL	24	val_24
\ No newline at end of file
+NULL	NULL	24	val_24
diff --git a/sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f
index eb0a6246c1988a18569f2ee01698c0532a3dfea8..0e75c1b63abb51c17fbd90e1ea9231d6addafc91 100644
--- a/sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -11,4 +11,4 @@
 19	val_19	19	val_19	NULL	NULL
 NULL	NULL	20	val_20	NULL	NULL
 NULL	NULL	24	val_24	NULL	NULL
-NULL	NULL	24	val_24	NULL	NULL
\ No newline at end of file
+NULL	NULL	24	val_24	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f
index f2e6e07d8fa1f554080391d4aca58acc50906056..c3f5359beb06fc543a4dddee2c173029752cd224 100644
--- a/sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -2,4 +2,4 @@
 12	val_12	NULL	NULL
 15	val_15	NULL	NULL
 11	val_11	NULL	NULL
-12	val_12	NULL	NULL
\ No newline at end of file
+12	val_12	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f
index 59821aeea3008af83bf6baa196e28b04461efcce..c9c79b85dfe24d2c5c9e7b278875729d91d6daff 100644
--- a/sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -1025,4 +1025,4 @@
 200	val_200
 200	val_200
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a b/sql/hive/src/test/resources/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 b/sql/hive/src/test/resources/golden/join_1to1-1-789b2636cfb6a08965e0bd190e419762
similarity index 100%
rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0
rename to sql/hive/src/test/resources/golden/join_1to1-1-789b2636cfb6a08965e0bd190e419762
diff --git a/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e b/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e
+++ b/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e
index 52a4d2c18e7018bc58ba76276d01bb661e8c8fd3..b57efb6ce27a200efe63013167aeab6a0923b289 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e
+++ b/sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e
@@ -27,4 +27,4 @@
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b
index fb5560e2d006c41a055fdd196c4759f7d45814c9..72e8fcf3ec4d0ccdd285b76407d86eee84c359fb 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b
+++ b/sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b
@@ -33,4 +33,4 @@ NULL	10050	66	NULL	NULL	NULL
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c
index 0ae4eb2d33595149228795c86dfd30876c166d81..0f21800b43be98db5c5755d10299daaac96d1dd1 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c
+++ b/sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c
@@ -31,4 +31,4 @@ NULL	10050	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 80	10040	88	NULL	NULL	NULL
-80	10040	88	NULL	NULL	NULL
\ No newline at end of file
+80	10040	88	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84
index fb5560e2d006c41a055fdd196c4759f7d45814c9..72e8fcf3ec4d0ccdd285b76407d86eee84c359fb 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84
+++ b/sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84
@@ -33,4 +33,4 @@ NULL	10050	66	NULL	NULL	NULL
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727
index 0ae4eb2d33595149228795c86dfd30876c166d81..0f21800b43be98db5c5755d10299daaac96d1dd1 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727
+++ b/sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727
@@ -31,4 +31,4 @@ NULL	10050	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 80	10040	88	NULL	NULL	NULL
-80	10040	88	NULL	NULL	NULL
\ No newline at end of file
+80	10040	88	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 b/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04
+++ b/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e
index 52a4d2c18e7018bc58ba76276d01bb661e8c8fd3..b57efb6ce27a200efe63013167aeab6a0923b289 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e
+++ b/sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e
@@ -27,4 +27,4 @@
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b
index fb5560e2d006c41a055fdd196c4759f7d45814c9..72e8fcf3ec4d0ccdd285b76407d86eee84c359fb 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b
+++ b/sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b
@@ -33,4 +33,4 @@ NULL	10050	66	NULL	NULL	NULL
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c
index 0ae4eb2d33595149228795c86dfd30876c166d81..0f21800b43be98db5c5755d10299daaac96d1dd1 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c
+++ b/sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c
@@ -31,4 +31,4 @@ NULL	10050	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 80	10040	88	NULL	NULL	NULL
-80	10040	88	NULL	NULL	NULL
\ No newline at end of file
+80	10040	88	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84
index fb5560e2d006c41a055fdd196c4759f7d45814c9..72e8fcf3ec4d0ccdd285b76407d86eee84c359fb 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84
+++ b/sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84
@@ -33,4 +33,4 @@ NULL	10050	66	NULL	NULL	NULL
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727
index 0ae4eb2d33595149228795c86dfd30876c166d81..0f21800b43be98db5c5755d10299daaac96d1dd1 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727
+++ b/sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727
@@ -31,4 +31,4 @@ NULL	10050	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 80	10040	88	NULL	NULL	NULL
-80	10040	88	NULL	NULL	NULL
\ No newline at end of file
+80	10040	88	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 b/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600
+++ b/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e
index 52a4d2c18e7018bc58ba76276d01bb661e8c8fd3..b57efb6ce27a200efe63013167aeab6a0923b289 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e
+++ b/sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e
@@ -27,4 +27,4 @@
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b
index fb5560e2d006c41a055fdd196c4759f7d45814c9..72e8fcf3ec4d0ccdd285b76407d86eee84c359fb 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b
+++ b/sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b
@@ -33,4 +33,4 @@ NULL	10050	66	NULL	NULL	NULL
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c
index 0ae4eb2d33595149228795c86dfd30876c166d81..0f21800b43be98db5c5755d10299daaac96d1dd1 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c
+++ b/sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c
@@ -31,4 +31,4 @@ NULL	10050	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 80	10040	88	NULL	NULL	NULL
-80	10040	88	NULL	NULL	NULL
\ No newline at end of file
+80	10040	88	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84
index fb5560e2d006c41a055fdd196c4759f7d45814c9..72e8fcf3ec4d0ccdd285b76407d86eee84c359fb 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84
+++ b/sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84
@@ -33,4 +33,4 @@ NULL	10050	66	NULL	NULL	NULL
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727
index 0ae4eb2d33595149228795c86dfd30876c166d81..0f21800b43be98db5c5755d10299daaac96d1dd1 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727
+++ b/sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727
@@ -31,4 +31,4 @@ NULL	10050	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 80	10040	88	NULL	NULL	NULL
-80	10040	88	NULL	NULL	NULL
\ No newline at end of file
+80	10040	88	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e b/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e
+++ b/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e
index 52a4d2c18e7018bc58ba76276d01bb661e8c8fd3..b57efb6ce27a200efe63013167aeab6a0923b289 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e
+++ b/sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e
@@ -27,4 +27,4 @@
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 b/sql/hive/src/test/resources/golden/join_1to1-3-6228e662e573a00ed04550d049d97a3b
similarity index 100%
rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9
rename to sql/hive/src/test/resources/golden/join_1to1-3-6228e662e573a00ed04550d049d97a3b
diff --git a/sql/hive/src/test/resources/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 b/sql/hive/src/test/resources/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b
index fb5560e2d006c41a055fdd196c4759f7d45814c9..72e8fcf3ec4d0ccdd285b76407d86eee84c359fb 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b
+++ b/sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b
@@ -33,4 +33,4 @@ NULL	10050	66	NULL	NULL	NULL
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c
index 0ae4eb2d33595149228795c86dfd30876c166d81..0f21800b43be98db5c5755d10299daaac96d1dd1 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c
+++ b/sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c
@@ -31,4 +31,4 @@ NULL	10050	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 80	10040	88	NULL	NULL	NULL
-80	10040	88	NULL	NULL	NULL
\ No newline at end of file
+80	10040	88	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84
index fb5560e2d006c41a055fdd196c4759f7d45814c9..72e8fcf3ec4d0ccdd285b76407d86eee84c359fb 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84
+++ b/sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84
@@ -33,4 +33,4 @@ NULL	10050	66	NULL	NULL	NULL
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727
index 0ae4eb2d33595149228795c86dfd30876c166d81..0f21800b43be98db5c5755d10299daaac96d1dd1 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727
+++ b/sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727
@@ -31,4 +31,4 @@ NULL	10050	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 80	10040	88	NULL	NULL	NULL
-80	10040	88	NULL	NULL	NULL
\ No newline at end of file
+80	10040	88	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 b/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04
+++ b/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e
index 52a4d2c18e7018bc58ba76276d01bb661e8c8fd3..b57efb6ce27a200efe63013167aeab6a0923b289 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e
+++ b/sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e
@@ -27,4 +27,4 @@
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b
index fb5560e2d006c41a055fdd196c4759f7d45814c9..72e8fcf3ec4d0ccdd285b76407d86eee84c359fb 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b
+++ b/sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b
@@ -33,4 +33,4 @@ NULL	10050	66	NULL	NULL	NULL
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c
index 0ae4eb2d33595149228795c86dfd30876c166d81..0f21800b43be98db5c5755d10299daaac96d1dd1 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c
+++ b/sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c
@@ -31,4 +31,4 @@ NULL	10050	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 80	10040	88	NULL	NULL	NULL
-80	10040	88	NULL	NULL	NULL
\ No newline at end of file
+80	10040	88	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84
index fb5560e2d006c41a055fdd196c4759f7d45814c9..72e8fcf3ec4d0ccdd285b76407d86eee84c359fb 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84
+++ b/sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84
@@ -33,4 +33,4 @@ NULL	10050	66	NULL	NULL	NULL
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727
index 0ae4eb2d33595149228795c86dfd30876c166d81..0f21800b43be98db5c5755d10299daaac96d1dd1 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727
+++ b/sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727
@@ -31,4 +31,4 @@ NULL	10050	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 80	10040	88	NULL	NULL	NULL
-80	10040	88	NULL	NULL	NULL
\ No newline at end of file
+80	10040	88	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 b/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600
+++ b/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e
index 52a4d2c18e7018bc58ba76276d01bb661e8c8fd3..b57efb6ce27a200efe63013167aeab6a0923b289 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e
+++ b/sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e
@@ -27,4 +27,4 @@
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b
index fb5560e2d006c41a055fdd196c4759f7d45814c9..72e8fcf3ec4d0ccdd285b76407d86eee84c359fb 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b
+++ b/sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b
@@ -33,4 +33,4 @@ NULL	10050	66	NULL	NULL	NULL
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c
index 0ae4eb2d33595149228795c86dfd30876c166d81..0f21800b43be98db5c5755d10299daaac96d1dd1 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c
+++ b/sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c
@@ -31,4 +31,4 @@ NULL	10050	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 80	10040	88	NULL	NULL	NULL
-80	10040	88	NULL	NULL	NULL
\ No newline at end of file
+80	10040	88	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84
index fb5560e2d006c41a055fdd196c4759f7d45814c9..72e8fcf3ec4d0ccdd285b76407d86eee84c359fb 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84
+++ b/sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84
@@ -33,4 +33,4 @@ NULL	10050	66	NULL	NULL	NULL
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727
index 0ae4eb2d33595149228795c86dfd30876c166d81..0f21800b43be98db5c5755d10299daaac96d1dd1 100644
--- a/sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727
+++ b/sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727
@@ -31,4 +31,4 @@ NULL	10050	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 70	10040	66	NULL	NULL	NULL
 80	10040	88	NULL	NULL	NULL
-80	10040	88	NULL	NULL	NULL
\ No newline at end of file
+80	10040	88	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 b/sql/hive/src/test/resources/golden/join_array-2-16840a0266cad03a1a0b134d105b854f
similarity index 100%
rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28
rename to sql/hive/src/test/resources/golden/join_array-2-16840a0266cad03a1a0b134d105b854f
diff --git a/sql/hive/src/test/resources/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de b/sql/hive/src/test/resources/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 b/sql/hive/src/test/resources/golden/join_array-3-a6ca6b64324596831033fdfe5b63a942
similarity index 100%
rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28
rename to sql/hive/src/test/resources/golden/join_array-3-a6ca6b64324596831033fdfe5b63a942
diff --git a/sql/hive/src/test/resources/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 b/sql/hive/src/test/resources/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 b/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62
index ce9688a065b34b8f3d59fee903967360b2f4c42f..84660b3eb4327b06e971ba997debb824614dd901 100644
--- a/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62
+++ b/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62
@@ -1 +1 @@
-10320092026892491	3312
\ No newline at end of file
+10320092026892491	3312
diff --git a/sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a b/sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a
index f937af554adef2236e6c19b88f84d37ebb8c3bec..1d1c48ae5a48fc2b537ff5e18179452929ca52a9 100644
--- a/sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a
+++ b/sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a
@@ -1,2 +1,2 @@
 10320092002467760	[0,23,37,48,53,55,55,56,60,66,72,76,77,78,80,81,87,88,90,90,91,90,92,97,100,103,104,107,108,108,109,110,113,113,113,113,113,113,114,116,116,116,117,116,117,117,117,115,115,117,117,117,121,120,131,131,131,125,125,124,124,128,128,131,131,132,133,134,134,134,134,26,26,null,null,null,null,116]
-10320092026892491	[0,2,59,106,131,142,159,244,320,398,417,433,553,616,710,826,917,971,1046,1051,1093,1112,1142,1215,1220,1226,1232,1267,1364,1549,1646,1948,2170,2272,2325,2433,2534,2852,2925,2992,3119,3207,3279,3323,3412,3637,3645,3634,3450,3473,3638,3688,3736,3758,3812,3862,3873,3868,3883,4118,4134,4127,4170,4216,null,null,null,null,3139]
\ No newline at end of file
+10320092026892491	[0,2,59,106,131,142,159,244,320,398,417,433,553,616,710,826,917,971,1046,1051,1093,1112,1142,1215,1220,1226,1232,1267,1364,1549,1646,1948,2170,2272,2325,2433,2534,2852,2925,2992,3119,3207,3279,3323,3412,3637,3645,3634,3450,3473,3638,3688,3736,3758,3812,3862,3873,3868,3883,4118,4134,4127,4170,4216,null,null,null,null,3139]
diff --git a/sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c b/sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c
index f937af554adef2236e6c19b88f84d37ebb8c3bec..1d1c48ae5a48fc2b537ff5e18179452929ca52a9 100644
--- a/sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c
+++ b/sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c
@@ -1,2 +1,2 @@
 10320092002467760	[0,23,37,48,53,55,55,56,60,66,72,76,77,78,80,81,87,88,90,90,91,90,92,97,100,103,104,107,108,108,109,110,113,113,113,113,113,113,114,116,116,116,117,116,117,117,117,115,115,117,117,117,121,120,131,131,131,125,125,124,124,128,128,131,131,132,133,134,134,134,134,26,26,null,null,null,null,116]
-10320092026892491	[0,2,59,106,131,142,159,244,320,398,417,433,553,616,710,826,917,971,1046,1051,1093,1112,1142,1215,1220,1226,1232,1267,1364,1549,1646,1948,2170,2272,2325,2433,2534,2852,2925,2992,3119,3207,3279,3323,3412,3637,3645,3634,3450,3473,3638,3688,3736,3758,3812,3862,3873,3868,3883,4118,4134,4127,4170,4216,null,null,null,null,3139]
\ No newline at end of file
+10320092026892491	[0,2,59,106,131,142,159,244,320,398,417,433,553,616,710,826,917,971,1046,1051,1093,1112,1142,1215,1220,1226,1232,1267,1364,1549,1646,1948,2170,2272,2325,2433,2534,2852,2925,2992,3119,3207,3279,3323,3412,3637,3645,3634,3450,3473,3638,3688,3736,3758,3812,3862,3873,3868,3883,4118,4134,4127,4170,4216,null,null,null,null,3139]
diff --git a/sql/hive/src/test/resources/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 b/sql/hive/src/test/resources/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 b/sql/hive/src/test/resources/golden/join_casesensitive-1-d1e9ae71a3ed691c39bb8f77ab28edbf
similarity index 100%
rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65
rename to sql/hive/src/test/resources/golden/join_casesensitive-1-d1e9ae71a3ed691c39bb8f77ab28edbf
diff --git a/sql/hive/src/test/resources/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc b/sql/hive/src/test/resources/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 b/sql/hive/src/test/resources/golden/join_casesensitive-3-93300f4a9242fa2804a5b368538d83f
similarity index 100%
rename from sql/hive/src/test/resources/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9
rename to sql/hive/src/test/resources/golden/join_casesensitive-3-93300f4a9242fa2804a5b368538d83f
diff --git a/sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd b/sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd
index 810f01f682c37e3945f1306b39b388496f6cf01a..f7d190a11d65c6b108836a38c163aa6f444e36d6 100644
--- a/sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd
+++ b/sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd
@@ -82,4 +82,4 @@ NULL	10050	66	50	10050	88
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
 80	10040	88	80	10040	66
-80	10040	88	80	10040	66
\ No newline at end of file
+80	10040	88	80	10040	66
diff --git a/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a b/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a
+++ b/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 b/sql/hive/src/test/resources/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d b/sql/hive/src/test/resources/golden/join_filters-1-bee6095f42de6a16708c2f9addc1b9bd
similarity index 100%
rename from sql/hive/src/test/resources/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d
rename to sql/hive/src/test/resources/golden/join_filters-1-bee6095f42de6a16708c2f9addc1b9bd
diff --git a/sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 b/sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01
+++ b/sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe b/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe
+++ b/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b b/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b
+++ b/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 b/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554
+++ b/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b b/sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b
+++ b/sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 b/sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0
+++ b/sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb b/sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb
+++ b/sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8 b/sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8
+++ b/sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 b/sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4
+++ b/sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e b/sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e
+++ b/sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 b/sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475
+++ b/sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 b/sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117
+++ b/sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba b/sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba
+++ b/sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 b/sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569
+++ b/sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 b/sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942
+++ b/sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba b/sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba
+++ b/sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 b/sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95
+++ b/sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 b/sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49
+++ b/sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 b/sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6
+++ b/sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 b/sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35
index b81820f88689445413313d381fbf0bb2cab9d7c2..1ef56d9d1d051446b57219bbbeb6206a0480427d 100644
--- a/sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35
+++ b/sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35
@@ -3,4 +3,4 @@ NULL	135	NULL	NULL
 48	NULL	NULL	NULL
 100	100	100	100
 148	NULL	NULL	NULL
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e b/sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e
index b81820f88689445413313d381fbf0bb2cab9d7c2..1ef56d9d1d051446b57219bbbeb6206a0480427d 100644
--- a/sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e
+++ b/sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e
@@ -3,4 +3,4 @@ NULL	135	NULL	NULL
 48	NULL	NULL	NULL
 100	100	100	100
 148	NULL	NULL	NULL
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a b/sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a
index b81820f88689445413313d381fbf0bb2cab9d7c2..1ef56d9d1d051446b57219bbbeb6206a0480427d 100644
--- a/sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a
+++ b/sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a
@@ -3,4 +3,4 @@ NULL	135	NULL	NULL
 48	NULL	NULL	NULL
 100	100	100	100
 148	NULL	NULL	NULL
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e b/sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e
+++ b/sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 b/sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709
index 71cd5ce0041b8ae72a4499df25b761eaded29180..768ebfc0854e45555aed9da8686d09bcf63a0db1 100644
--- a/sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709
+++ b/sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709
@@ -3,4 +3,4 @@ NULL	NULL	NULL	135
 NULL	NULL	48	NULL
 NULL	NULL	148	NULL
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada b/sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada
index 71cd5ce0041b8ae72a4499df25b761eaded29180..768ebfc0854e45555aed9da8686d09bcf63a0db1 100644
--- a/sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada
+++ b/sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada
@@ -3,4 +3,4 @@ NULL	NULL	NULL	135
 NULL	NULL	48	NULL
 NULL	NULL	148	NULL
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 b/sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98
index 71cd5ce0041b8ae72a4499df25b761eaded29180..768ebfc0854e45555aed9da8686d09bcf63a0db1 100644
--- a/sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98
+++ b/sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98
@@ -3,4 +3,4 @@ NULL	NULL	NULL	135
 NULL	NULL	48	NULL
 NULL	NULL	148	NULL
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 b/sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52
+++ b/sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf b/sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf
+++ b/sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 b/sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046
+++ b/sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 b/sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622
+++ b/sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 b/sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940
+++ b/sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 b/sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65
index 24a0c79d42923487da930766ff9b567bc1aa7a59..3414d863630932524604ff75a9daa517b6c996d0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65
+++ b/sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65
@@ -4,4 +4,4 @@ NULL	NULL	48	NULL
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f b/sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f
index 24a0c79d42923487da930766ff9b567bc1aa7a59..3414d863630932524604ff75a9daa517b6c996d0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f
+++ b/sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f
@@ -4,4 +4,4 @@ NULL	NULL	48	NULL
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 b/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496
+++ b/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf b/sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf
index 24a0c79d42923487da930766ff9b567bc1aa7a59..3414d863630932524604ff75a9daa517b6c996d0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf
+++ b/sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf
@@ -4,4 +4,4 @@ NULL	NULL	48	NULL
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e b/sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e
index 24a0c79d42923487da930766ff9b567bc1aa7a59..3414d863630932524604ff75a9daa517b6c996d0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e
+++ b/sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e
@@ -4,4 +4,4 @@ NULL	NULL	48	NULL
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e b/sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e
index d735f349a9542983cbbb617519bdbb352287a4ea..db3f1d99ec2613de8779fb5355de5e4d9eec35dd 100644
--- a/sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e
+++ b/sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	NULL	NULL	40
 NULL	NULL	NULL	NULL	12	35
 NULL	NULL	NULL	NULL	48	NULL
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f b/sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f
index ea646edf6d143e92f524609dd83b63455f26c81f..04ecda59b21bd00055e108ffc064565574917de1 100644
--- a/sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f
+++ b/sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40	NULL	NULL
 NULL	NULL	12	35	NULL	NULL
 NULL	NULL	48	NULL	NULL	NULL
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 b/sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113
index d735f349a9542983cbbb617519bdbb352287a4ea..db3f1d99ec2613de8779fb5355de5e4d9eec35dd 100644
--- a/sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113
+++ b/sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	NULL	NULL	40
 NULL	NULL	NULL	NULL	12	35
 NULL	NULL	NULL	NULL	48	NULL
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 b/sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3
index d735f349a9542983cbbb617519bdbb352287a4ea..db3f1d99ec2613de8779fb5355de5e4d9eec35dd 100644
--- a/sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3
+++ b/sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	NULL	NULL	40
 NULL	NULL	NULL	NULL	12	35
 NULL	NULL	NULL	NULL	48	NULL
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca b/sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca
index ea646edf6d143e92f524609dd83b63455f26c81f..04ecda59b21bd00055e108ffc064565574917de1 100644
--- a/sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca
+++ b/sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40	NULL	NULL
 NULL	NULL	12	35	NULL	NULL
 NULL	NULL	48	NULL	NULL	NULL
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 b/sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242
index d735f349a9542983cbbb617519bdbb352287a4ea..db3f1d99ec2613de8779fb5355de5e4d9eec35dd 100644
--- a/sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242
+++ b/sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	NULL	NULL	40
 NULL	NULL	NULL	NULL	12	35
 NULL	NULL	NULL	NULL	48	NULL
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 b/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359
+++ b/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 b/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35
+++ b/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 b/sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7
+++ b/sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf b/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf
+++ b/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f b/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f
+++ b/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab b/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab
+++ b/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 b/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027
+++ b/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 b/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9
+++ b/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe b/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe
+++ b/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b b/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b
+++ b/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 b/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554
+++ b/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b b/sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b
+++ b/sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 b/sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0
+++ b/sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 b/sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757
+++ b/sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb b/sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb
+++ b/sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8 b/sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8
+++ b/sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 b/sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4
+++ b/sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e b/sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e
+++ b/sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f b/sql/hive/src/test/resources/golden/join_filters-46-268d8fb3cb9b04eb269fe7ec40a24dfe
similarity index 100%
rename from sql/hive/src/test/resources/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f
rename to sql/hive/src/test/resources/golden/join_filters-46-268d8fb3cb9b04eb269fe7ec40a24dfe
diff --git a/sql/hive/src/test/resources/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b b/sql/hive/src/test/resources/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 b/sql/hive/src/test/resources/golden/join_filters-47-3c52df82c7d78501610f3f898103f753
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 b/sql/hive/src/test/resources/golden/join_filters-47-6dc6866a65c74d69538b776b41b06c16
similarity index 100%
rename from sql/hive/src/test/resources/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65
rename to sql/hive/src/test/resources/golden/join_filters-47-6dc6866a65c74d69538b776b41b06c16
diff --git a/sql/hive/src/test/resources/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c b/sql/hive/src/test/resources/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 b/sql/hive/src/test/resources/golden/join_filters-48-e884480a0f7273d3e2f2de2ba46b855c
similarity index 100%
rename from sql/hive/src/test/resources/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9
rename to sql/hive/src/test/resources/golden/join_filters-48-e884480a0f7273d3e2f2de2ba46b855c
diff --git a/sql/hive/src/test/resources/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 b/sql/hive/src/test/resources/golden/join_filters-49-98fd86aea9cacaa82d43c7468109dd33
similarity index 100%
rename from sql/hive/src/test/resources/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2
rename to sql/hive/src/test/resources/golden/join_filters-49-98fd86aea9cacaa82d43c7468109dd33
diff --git a/sql/hive/src/test/resources/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 b/sql/hive/src/test/resources/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e b/sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e
index 24a0c79d42923487da930766ff9b567bc1aa7a59..3414d863630932524604ff75a9daa517b6c996d0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e
+++ b/sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e
@@ -4,4 +4,4 @@ NULL	NULL	48	NULL
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1 b/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1
+++ b/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac b/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac
+++ b/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 b/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71
+++ b/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 b/sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475
+++ b/sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba b/sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba
+++ b/sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 b/sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569
+++ b/sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 b/sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942
+++ b/sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba b/sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba
+++ b/sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 b/sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95
+++ b/sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 b/sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49
+++ b/sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 b/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9
+++ b/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 b/sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6
+++ b/sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 b/sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35
index b81820f88689445413313d381fbf0bb2cab9d7c2..1ef56d9d1d051446b57219bbbeb6206a0480427d 100644
--- a/sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35
+++ b/sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35
@@ -3,4 +3,4 @@ NULL	135	NULL	NULL
 48	NULL	NULL	NULL
 100	100	100	100
 148	NULL	NULL	NULL
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e b/sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e
index b81820f88689445413313d381fbf0bb2cab9d7c2..1ef56d9d1d051446b57219bbbeb6206a0480427d 100644
--- a/sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e
+++ b/sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e
@@ -3,4 +3,4 @@ NULL	135	NULL	NULL
 48	NULL	NULL	NULL
 100	100	100	100
 148	NULL	NULL	NULL
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a b/sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a
index b81820f88689445413313d381fbf0bb2cab9d7c2..1ef56d9d1d051446b57219bbbeb6206a0480427d 100644
--- a/sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a
+++ b/sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a
@@ -3,4 +3,4 @@ NULL	135	NULL	NULL
 48	NULL	NULL	NULL
 100	100	100	100
 148	NULL	NULL	NULL
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 b/sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709
index 71cd5ce0041b8ae72a4499df25b761eaded29180..768ebfc0854e45555aed9da8686d09bcf63a0db1 100644
--- a/sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709
+++ b/sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709
@@ -3,4 +3,4 @@ NULL	NULL	NULL	135
 NULL	NULL	48	NULL
 NULL	NULL	148	NULL
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada b/sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada
index 71cd5ce0041b8ae72a4499df25b761eaded29180..768ebfc0854e45555aed9da8686d09bcf63a0db1 100644
--- a/sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada
+++ b/sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada
@@ -3,4 +3,4 @@ NULL	NULL	NULL	135
 NULL	NULL	48	NULL
 NULL	NULL	148	NULL
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 b/sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98
index 71cd5ce0041b8ae72a4499df25b761eaded29180..768ebfc0854e45555aed9da8686d09bcf63a0db1 100644
--- a/sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98
+++ b/sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98
@@ -3,4 +3,4 @@ NULL	NULL	NULL	135
 NULL	NULL	48	NULL
 NULL	NULL	148	NULL
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 b/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496
+++ b/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 b/sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7
+++ b/sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 b/sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757
+++ b/sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d b/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d
+++ b/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e b/sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e
index 24a0c79d42923487da930766ff9b567bc1aa7a59..3414d863630932524604ff75a9daa517b6c996d0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e
+++ b/sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e
@@ -4,4 +4,4 @@ NULL	NULL	48	NULL
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 b/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9
+++ b/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d b/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d
+++ b/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 b/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5
+++ b/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa b/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa
+++ b/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 b/sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01
+++ b/sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 b/sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117
+++ b/sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e b/sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e
+++ b/sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 b/sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52
index 51a29d71030eecb780500a5c843cc9960f08bb18..a8dc3bebed62c79944d02cbc47f6f2b0d314cfca 100644
--- a/sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52
+++ b/sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52
@@ -1,4 +1,4 @@
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf b/sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf
+++ b/sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 b/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5
+++ b/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 b/sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046
+++ b/sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 b/sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622
+++ b/sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 b/sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940
index 759113b3f1c96dfe24c5e5cbfb44e65863a4baab..0fd19b648c18547469ce595eb602a860194190b2 100644
--- a/sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940
+++ b/sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40
 NULL	NULL	12	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 b/sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65
index 24a0c79d42923487da930766ff9b567bc1aa7a59..3414d863630932524604ff75a9daa517b6c996d0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65
+++ b/sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65
@@ -4,4 +4,4 @@ NULL	NULL	48	NULL
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f b/sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f
index 24a0c79d42923487da930766ff9b567bc1aa7a59..3414d863630932524604ff75a9daa517b6c996d0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f
+++ b/sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f
@@ -4,4 +4,4 @@ NULL	NULL	48	NULL
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf b/sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf
index 24a0c79d42923487da930766ff9b567bc1aa7a59..3414d863630932524604ff75a9daa517b6c996d0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf
+++ b/sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf
@@ -4,4 +4,4 @@ NULL	NULL	48	NULL
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e b/sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e
index 24a0c79d42923487da930766ff9b567bc1aa7a59..3414d863630932524604ff75a9daa517b6c996d0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e
+++ b/sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e
@@ -4,4 +4,4 @@ NULL	NULL	48	NULL
 NULL	40	NULL	NULL
 12	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e b/sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e
index d735f349a9542983cbbb617519bdbb352287a4ea..db3f1d99ec2613de8779fb5355de5e4d9eec35dd 100644
--- a/sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e
+++ b/sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	NULL	NULL	40
 NULL	NULL	NULL	NULL	12	35
 NULL	NULL	NULL	NULL	48	NULL
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f b/sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f
index ea646edf6d143e92f524609dd83b63455f26c81f..04ecda59b21bd00055e108ffc064565574917de1 100644
--- a/sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f
+++ b/sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40	NULL	NULL
 NULL	NULL	12	35	NULL	NULL
 NULL	NULL	48	NULL	NULL	NULL
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 b/sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113
index d735f349a9542983cbbb617519bdbb352287a4ea..db3f1d99ec2613de8779fb5355de5e4d9eec35dd 100644
--- a/sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113
+++ b/sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	NULL	NULL	40
 NULL	NULL	NULL	NULL	12	35
 NULL	NULL	NULL	NULL	48	NULL
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa b/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa
+++ b/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 b/sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3
index d735f349a9542983cbbb617519bdbb352287a4ea..db3f1d99ec2613de8779fb5355de5e4d9eec35dd 100644
--- a/sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3
+++ b/sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	NULL	NULL	40
 NULL	NULL	NULL	NULL	12	35
 NULL	NULL	NULL	NULL	48	NULL
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca b/sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca
index ea646edf6d143e92f524609dd83b63455f26c81f..04ecda59b21bd00055e108ffc064565574917de1 100644
--- a/sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca
+++ b/sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	40	NULL	NULL
 NULL	NULL	12	35	NULL	NULL
 NULL	NULL	48	NULL	NULL	NULL
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 b/sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242
index d735f349a9542983cbbb617519bdbb352287a4ea..db3f1d99ec2613de8779fb5355de5e4d9eec35dd 100644
--- a/sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242
+++ b/sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	NULL	NULL	40
 NULL	NULL	NULL	NULL	12	35
 NULL	NULL	NULL	NULL	48	NULL
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 b/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359
+++ b/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 b/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35
+++ b/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf b/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf
+++ b/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f b/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f
+++ b/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab b/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab
+++ b/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 b/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027
+++ b/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 b/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9
+++ b/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce b/sql/hive/src/test/resources/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 b/sql/hive/src/test/resources/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 b/sql/hive/src/test/resources/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 b/sql/hive/src/test/resources/golden/join_hive_626-3-4a2f2f2858540afea9a195b5322941ee
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6
rename to sql/hive/src/test/resources/golden/join_hive_626-3-4a2f2f2858540afea9a195b5322941ee
diff --git a/sql/hive/src/test/resources/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 b/sql/hive/src/test/resources/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/escape1-3-4267651148da591da38737028fdbd80 b/sql/hive/src/test/resources/golden/join_hive_626-4-4bb73b33747da4ed852df381b7b45a71
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape1-3-4267651148da591da38737028fdbd80
rename to sql/hive/src/test/resources/golden/join_hive_626-4-4bb73b33747da4ed852df381b7b45a71
diff --git a/sql/hive/src/test/resources/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 b/sql/hive/src/test/resources/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 b/sql/hive/src/test/resources/golden/join_hive_626-5-c6a3ae6f3539ab48b996060fb51d8ebe
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62
rename to sql/hive/src/test/resources/golden/join_hive_626-5-c6a3ae6f3539ab48b996060fb51d8ebe
diff --git a/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 b/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2
index e3143e49c2badb8057554888961f127df6a85dac..8b00ef084005b73927434be476810925764d2c4d 100644
--- a/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2
+++ b/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2
@@ -1 +1 @@
-foo1	bar10	2
\ No newline at end of file
+foo1	bar10	2
diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa
index 4c88927e5657907921a5ce35966bd309a7b28c27..120056ea10c6029591a4be60fe32c14a8306bb15 100644
--- a/sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -104,4 +104,4 @@
 98	val_98	val_98
 98	val_98	val_98
 98	val_98	val_98
-98	val_98	val_98
\ No newline at end of file
+98	val_98	val_98
diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa
index 4c88927e5657907921a5ce35966bd309a7b28c27..120056ea10c6029591a4be60fe32c14a8306bb15 100644
--- a/sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa
+++ b/sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa
@@ -104,4 +104,4 @@
 98	val_98	val_98
 98	val_98	val_98
 98	val_98	val_98
-98	val_98	val_98
\ No newline at end of file
+98	val_98	val_98
diff --git a/sql/hive/src/test/resources/golden/escape1-5-70729c3d79ded87e884c176138174645 b/sql/hive/src/test/resources/golden/join_nulls-1-75b1f5331b62fedb7dbbe6ac93a3c83f
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape1-5-70729c3d79ded87e884c176138174645
rename to sql/hive/src/test/resources/golden/join_nulls-1-75b1f5331b62fedb7dbbe6ac93a3c83f
diff --git a/sql/hive/src/test/resources/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 b/sql/hive/src/test/resources/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b b/sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b
index 4bc2d3969f17fca4cf1d611c99223ff935d8c97a..610f43b2ac6dc42df1a1cd178b99b379fd19573f 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b
+++ b/sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b
@@ -1,3 +1,3 @@
 NULL	35	NULL	35
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 b/sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8
index 8414cfbede40af67e573dd07bfa6723745213bb2..19621e45f37580f62292cfea14a2840b4abcd29e 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8
+++ b/sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8
@@ -1,3 +1,3 @@
 NULL	35	NULL	NULL
 48	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 b/sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062
index c11ae6cce7abb67194827accf86fbba7a4ee0934..b512581d48dfe4ab1348e5a0a8e6ec9478549174 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062
+++ b/sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062
@@ -1,3 +1,3 @@
 NULL	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d b/sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d
index e28e1b1e5e8b144f61f2305717f5d95feec23376..ee5b5e1ec30961ce04561744666994a9c70af394 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d
+++ b/sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d
@@ -1,3 +1,3 @@
 NULL	NULL	NULL	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 b/sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077
index 85192b86801e2f63abc349b97b9dbed7b6c95688..9a862c2d08e7c60ad114f9c6a6a0446be030daa8 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077
+++ b/sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077
@@ -1,3 +1,3 @@
 NULL	NULL	NULL	35
 48	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 b/sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126
index c19237d5d4309832e35ba5b6ab2ca88acfd2024c..5d0c8eb89e421e1db425e9a1ef0be2be7b699626 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126
+++ b/sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126
@@ -1,3 +1,3 @@
 NULL	NULL	48	NULL
 NULL	35	NULL	35
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 b/sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9
index e28e1b1e5e8b144f61f2305717f5d95feec23376..ee5b5e1ec30961ce04561744666994a9c70af394 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9
+++ b/sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9
@@ -1,3 +1,3 @@
 NULL	NULL	NULL	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f b/sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f
index 56ca29ffe263c6429be8837f9be1eba9a950e661..25db723b063786c6dc6b70763d5008d228dfe460 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f
+++ b/sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f
@@ -2,4 +2,4 @@ NULL	NULL	NULL	35
 NULL	NULL	48	NULL
 NULL	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 b/sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2
index 92e228d381628717f0076f97bc6788e076a9c217..810287110dc402d589d1a933670b3d3ee98f2434 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2
+++ b/sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2
@@ -1,4 +1,4 @@
 NULL	NULL	NULL	35
 NULL	35	NULL	NULL
 48	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c b/sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c
index f57bbd51528524e3cca9d63ba9e2f2ac45ac4268..115f8d9531800568d555217581340629a5a5faca 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c
+++ b/sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c
@@ -1,4 +1,4 @@
 NULL	NULL	48	NULL
 NULL	35	NULL	35
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 b/sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5
index 0d14bf4e3850930ab8d4177a4b5d49124d088c91..bb1fee6e62e994512d764d01bb78d03a1f13477a 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5
+++ b/sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5
@@ -6,4 +6,4 @@ NULL	35	100	100
 48	NULL	100	100
 100	100	NULL	35
 100	100	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 b/sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5
index 56ca29ffe263c6429be8837f9be1eba9a950e661..25db723b063786c6dc6b70763d5008d228dfe460 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5
+++ b/sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5
@@ -2,4 +2,4 @@ NULL	NULL	NULL	35
 NULL	NULL	48	NULL
 NULL	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e b/sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e
index 0db9ffd61a3adf0ebbf7688ac004c79e73a23c7d..9bca4244a2eb4a313d3343028ec6878311db4a5b 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e
+++ b/sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e
@@ -1,3 +1,3 @@
 NULL	NULL	NULL	NULL	48	NULL
 NULL	35	NULL	35	NULL	35
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 b/sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0
index ed1b35fbe80ae3f5228d0b4ec96626bd7debed9f..f7ff69b919598c7b24668584a9836853eb68e876 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0
+++ b/sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0
@@ -1,3 +1,3 @@
 NULL	NULL	48	NULL	NULL	NULL
 NULL	35	NULL	35	NULL	35
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 b/sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8
index 0db9ffd61a3adf0ebbf7688ac004c79e73a23c7d..9bca4244a2eb4a313d3343028ec6878311db4a5b 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8
+++ b/sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8
@@ -1,3 +1,3 @@
 NULL	NULL	NULL	NULL	48	NULL
 NULL	35	NULL	35	NULL	35
-100	100	100	100	100	100
\ No newline at end of file
+100	100	100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 b/sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495
index 0d14bf4e3850930ab8d4177a4b5d49124d088c91..bb1fee6e62e994512d764d01bb78d03a1f13477a 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495
+++ b/sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495
@@ -6,4 +6,4 @@ NULL	35	100	100
 48	NULL	100	100
 100	100	NULL	35
 100	100	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 b/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56
+++ b/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 b/sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9
index bdd09da460bbed9173dde7d3a8d9bb4d423dfd87..309eec4c7d309091d533da8153a0a8c1c3effdcd 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9
+++ b/sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9
@@ -1,2 +1,2 @@
 48	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 b/sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83
index dfe4f4318fc66e1083d81ff8f8ed70f4ddb6aa6b..96e8fb0241578f6b0e8bafd72ffb9d17c88c133c 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83
+++ b/sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83
@@ -1,2 +1,2 @@
 NULL	35	NULL	35
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb b/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb
+++ b/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb b/sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb
index bdd09da460bbed9173dde7d3a8d9bb4d423dfd87..309eec4c7d309091d533da8153a0a8c1c3effdcd 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb
+++ b/sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb
@@ -1,2 +1,2 @@
 48	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 b/sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4
index 0d14bf4e3850930ab8d4177a4b5d49124d088c91..bb1fee6e62e994512d764d01bb78d03a1f13477a 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4
+++ b/sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4
@@ -6,4 +6,4 @@ NULL	35	100	100
 48	NULL	100	100
 100	100	NULL	35
 100	100	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 b/sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1
index dfe4f4318fc66e1083d81ff8f8ed70f4ddb6aa6b..96e8fb0241578f6b0e8bafd72ffb9d17c88c133c 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1
+++ b/sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1
@@ -1,2 +1,2 @@
 NULL	35	NULL	35
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 b/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486
+++ b/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 b/sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352
index c11ae6cce7abb67194827accf86fbba7a4ee0934..b512581d48dfe4ab1348e5a0a8e6ec9478549174 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352
+++ b/sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352
@@ -1,3 +1,3 @@
 NULL	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 b/sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26
index 8414cfbede40af67e573dd07bfa6723745213bb2..19621e45f37580f62292cfea14a2840b4abcd29e 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26
+++ b/sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26
@@ -1,3 +1,3 @@
 NULL	35	NULL	NULL
 48	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 b/sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1
index 4bc2d3969f17fca4cf1d611c99223ff935d8c97a..610f43b2ac6dc42df1a1cd178b99b379fd19573f 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1
+++ b/sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1
@@ -1,3 +1,3 @@
 NULL	35	NULL	35
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 b/sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82
index e28e1b1e5e8b144f61f2305717f5d95feec23376..ee5b5e1ec30961ce04561744666994a9c70af394 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82
+++ b/sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82
@@ -1,3 +1,3 @@
 NULL	NULL	NULL	35
 NULL	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 b/sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257
index 85192b86801e2f63abc349b97b9dbed7b6c95688..9a862c2d08e7c60ad114f9c6a6a0446be030daa8 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257
+++ b/sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257
@@ -1,3 +1,3 @@
 NULL	NULL	NULL	35
 48	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc b/sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc
index c19237d5d4309832e35ba5b6ab2ca88acfd2024c..5d0c8eb89e421e1db425e9a1ef0be2be7b699626 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc
+++ b/sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc
@@ -1,3 +1,3 @@
 NULL	NULL	48	NULL
 NULL	35	NULL	35
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 b/sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94
index 0d14bf4e3850930ab8d4177a4b5d49124d088c91..bb1fee6e62e994512d764d01bb78d03a1f13477a 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94
+++ b/sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94
@@ -6,4 +6,4 @@ NULL	35	100	100
 48	NULL	100	100
 100	100	NULL	35
 100	100	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 b/sql/hive/src/test/resources/golden/join_nulls-40-268d8fb3cb9b04eb269fe7ec40a24dfe
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0
rename to sql/hive/src/test/resources/golden/join_nulls-40-268d8fb3cb9b04eb269fe7ec40a24dfe
diff --git a/sql/hive/src/test/resources/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b b/sql/hive/src/test/resources/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 b/sql/hive/src/test/resources/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e b/sql/hive/src/test/resources/golden/join_nulls-41-6dc6866a65c74d69538b776b41b06c16
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e
rename to sql/hive/src/test/resources/golden/join_nulls-41-6dc6866a65c74d69538b776b41b06c16
diff --git a/sql/hive/src/test/resources/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c b/sql/hive/src/test/resources/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe b/sql/hive/src/test/resources/golden/join_nulls-42-e884480a0f7273d3e2f2de2ba46b855c
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe
rename to sql/hive/src/test/resources/golden/join_nulls-42-e884480a0f7273d3e2f2de2ba46b855c
diff --git a/sql/hive/src/test/resources/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 b/sql/hive/src/test/resources/golden/join_nulls-43-98fd86aea9cacaa82d43c7468109dd33
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800
rename to sql/hive/src/test/resources/golden/join_nulls-43-98fd86aea9cacaa82d43c7468109dd33
diff --git a/sql/hive/src/test/resources/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 b/sql/hive/src/test/resources/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 b/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412
+++ b/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 b/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9
+++ b/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 b/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71
+++ b/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 b/sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1
index 7d9efd15fa28770c7722baa97aac9bac668b20a5..c823f539ee32856fed98b05553a4e640264e8a17 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1
+++ b/sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1
@@ -1,4 +1,4 @@
 48	NULL	48	NULL
 100	100	100	100
 148	NULL	148	NULL
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 b/sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3
+++ b/sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 b/sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7
index 4d2ff6a2374160b71b7dcb5f49d1211f51120b72..2b06d1c29d9846980495a954dbcc0b58d87c2ca8 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7
+++ b/sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7
@@ -3,4 +3,4 @@ NULL	NULL	NULL	135
 48	NULL	48	NULL
 100	100	100	100
 148	NULL	148	NULL
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c b/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c
+++ b/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 b/sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470
index 7d9efd15fa28770c7722baa97aac9bac668b20a5..c823f539ee32856fed98b05553a4e640264e8a17 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470
+++ b/sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470
@@ -1,4 +1,4 @@
 48	NULL	48	NULL
 100	100	100	100
 148	NULL	148	NULL
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f b/sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f
index 695bb1544ec89241c30da1dd52e1aa4aca1de709..06f37e109247c71246c58c189d3429b05cb0a82f 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f
+++ b/sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f
@@ -3,4 +3,4 @@ NULL	135	NULL	NULL
 48	NULL	48	NULL
 100	100	100	100
 148	NULL	148	NULL
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 b/sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564
+++ b/sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 b/sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363
index b89f5e6d42a0f5ffba0b3f8b5a99fafeeea406e0..340baa63de2f720de5fd7cefec58f5dace8b4596 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363
+++ b/sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363
@@ -1,2 +1,2 @@
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 b/sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502
index b81820f88689445413313d381fbf0bb2cab9d7c2..1ef56d9d1d051446b57219bbbeb6206a0480427d 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502
+++ b/sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502
@@ -3,4 +3,4 @@ NULL	135	NULL	NULL
 48	NULL	NULL	NULL
 100	100	100	100
 148	NULL	NULL	NULL
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 b/sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564
index 71cd5ce0041b8ae72a4499df25b761eaded29180..768ebfc0854e45555aed9da8686d09bcf63a0db1 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564
+++ b/sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564
@@ -3,4 +3,4 @@ NULL	NULL	NULL	135
 NULL	NULL	48	NULL
 NULL	NULL	148	NULL
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 b/sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178
index 86b3dab6bffc8b3cb21a86ae572bc91daa642a51..88b1f5331b84e2c8e36eccb4b12828a7e7d5b098 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178
+++ b/sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178
@@ -1,4 +1,4 @@
 NULL	35	NULL	35
 NULL	135	NULL	135
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 b/sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158
index 169df31887b9647fc6843c728b584c4225407214..b060f40ffdf48692ca727337a36203bd313e2df0 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158
+++ b/sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158
@@ -3,4 +3,4 @@ NULL	NULL	148	NULL
 NULL	35	NULL	35
 NULL	135	NULL	135
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a b/sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a
index 86b3dab6bffc8b3cb21a86ae572bc91daa642a51..88b1f5331b84e2c8e36eccb4b12828a7e7d5b098 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a
+++ b/sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a
@@ -1,4 +1,4 @@
 NULL	35	NULL	35
 NULL	135	NULL	135
 100	100	100	100
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 b/sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24
index bdd5dcf63fd005bd9d9b31f1167b44762253757a..1256e9451914b7277fdf6b81b6edeb822ea585ab 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24
+++ b/sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24
@@ -3,4 +3,4 @@ NULL	135	NULL	135
 48	NULL	NULL	NULL
 100	100	100	100
 148	NULL	NULL	NULL
-200	200	200	200
\ No newline at end of file
+200	200	200	200
diff --git a/sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 b/sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67
index bdd09da460bbed9173dde7d3a8d9bb4d423dfd87..309eec4c7d309091d533da8153a0a8c1c3effdcd 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67
+++ b/sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67
@@ -1,2 +1,2 @@
 48	NULL	48	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 b/sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0
index dfe4f4318fc66e1083d81ff8f8ed70f4ddb6aa6b..96e8fb0241578f6b0e8bafd72ffb9d17c88c133c 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0
+++ b/sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0
@@ -1,2 +1,2 @@
 NULL	35	NULL	35
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 b/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870
index 6dc85dd550540f066c26ccb834da42e4c6596203..46a059f07aca66f64ada4091705a2d20cf8bc2f0 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870
+++ b/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870
@@ -1 +1 @@
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b b/sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b
index c11ae6cce7abb67194827accf86fbba7a4ee0934..b512581d48dfe4ab1348e5a0a8e6ec9478549174 100644
--- a/sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b
+++ b/sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b
@@ -1,3 +1,3 @@
 NULL	35	NULL	NULL
 48	NULL	NULL	NULL
-100	100	100	100
\ No newline at end of file
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 b/sql/hive/src/test/resources/golden/join_nullsafe-2-5bb63fafa390b1d4c20e225a8a648dcf
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62
rename to sql/hive/src/test/resources/golden/join_nullsafe-2-5bb63fafa390b1d4c20e225a8a648dcf
diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 b/sql/hive/src/test/resources/golden/join_nullsafe-2-793e288c9e0971f0bf3f37493f76dc7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/escape2-7-70729c3d79ded87e884c176138174645 b/sql/hive/src/test/resources/golden/join_nullsafe-20-88faf8a93ba6759bd6f2bbcbdcfecda0
similarity index 100%
rename from sql/hive/src/test/resources/golden/escape2-7-70729c3d79ded87e884c176138174645
rename to sql/hive/src/test/resources/golden/join_nullsafe-20-88faf8a93ba6759bd6f2bbcbdcfecda0
diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-20-d6fc260320c577eec9a5db0d4135d224 b/sql/hive/src/test/resources/golden/join_nullsafe-20-d6fc260320c577eec9a5db0d4135d224
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/join_nullsafe-21-24332b9390108fb3379e1acc599293a1
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77
rename to sql/hive/src/test/resources/golden/join_nullsafe-21-24332b9390108fb3379e1acc599293a1
diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-21-a60dae725ffc543f805242611d99de4e b/sql/hive/src/test/resources/golden/join_nullsafe-21-a60dae725ffc543f805242611d99de4e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-22-24c80d0f9e3d72c48d947770fa184985 b/sql/hive/src/test/resources/golden/join_nullsafe-22-24c80d0f9e3d72c48d947770fa184985
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/join_nullsafe-22-4be80634a6bd916e3ebd60a124f0a48e
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77
rename to sql/hive/src/test/resources/golden/join_nullsafe-22-4be80634a6bd916e3ebd60a124f0a48e
diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-23-3fe6ae20cab3417759dcc654a3a26746 b/sql/hive/src/test/resources/golden/join_nullsafe-23-3fe6ae20cab3417759dcc654a3a26746
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/join_nullsafe-23-e4425d56be43c21124d95160653ce0ac
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd
rename to sql/hive/src/test/resources/golden/join_nullsafe-23-e4425d56be43c21124d95160653ce0ac
diff --git a/sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 b/sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263
index 59821aeea3008af83bf6baa196e28b04461efcce..c9c79b85dfe24d2c5c9e7b278875729d91d6daff 100644
--- a/sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263
+++ b/sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263
@@ -1025,4 +1025,4 @@
 200	val_200
 200	val_200
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 b/sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564
index 06a83e95d4f79d31231238445463f21d587de8bb..33e6a1546ca65df70809a95a187180f11f2b3343 100644
--- a/sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564
+++ b/sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564
@@ -1 +1 @@
-2	12	2	22	2	12	2	12
\ No newline at end of file
+2	12	2	22	2	12	2	12
diff --git a/sql/hive/src/test/resources/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/join_reorder2-4-c0f14def6a135cc50cba364e810ce28e
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/join_reorder2-4-c0f14def6a135cc50cba364e810ce28e
diff --git a/sql/hive/src/test/resources/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/join_reorder2-5-ade68a23d7b1a4f328623bb5a0f07488
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/join_reorder2-5-ade68a23d7b1a4f328623bb5a0f07488
diff --git a/sql/hive/src/test/resources/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/join_reorder2-6-8eb53fb8f05a43ee377aa1c927857e7c
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc
rename to sql/hive/src/test/resources/golden/join_reorder2-6-8eb53fb8f05a43ee377aa1c927857e7c
diff --git a/sql/hive/src/test/resources/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec b/sql/hive/src/test/resources/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/join_reorder2-7-5f4cfbbe53c5e808ee08b26514272034
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77
rename to sql/hive/src/test/resources/golden/join_reorder2-7-5f4cfbbe53c5e808ee08b26514272034
diff --git a/sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 b/sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828
index 06a83e95d4f79d31231238445463f21d587de8bb..33e6a1546ca65df70809a95a187180f11f2b3343 100644
--- a/sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828
+++ b/sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828
@@ -1 +1 @@
-2	12	2	22	2	12	2	12
\ No newline at end of file
+2	12	2	22	2	12	2	12
diff --git a/sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 b/sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583
index 06a83e95d4f79d31231238445463f21d587de8bb..33e6a1546ca65df70809a95a187180f11f2b3343 100644
--- a/sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583
+++ b/sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583
@@ -1 +1 @@
-2	12	2	22	2	12	2	12
\ No newline at end of file
+2	12	2	22	2	12	2	12
diff --git a/sql/hive/src/test/resources/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/join_reorder3-4-c0f14def6a135cc50cba364e810ce28e
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd
rename to sql/hive/src/test/resources/golden/join_reorder3-4-c0f14def6a135cc50cba364e810ce28e
diff --git a/sql/hive/src/test/resources/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/join_reorder3-5-ade68a23d7b1a4f328623bb5a0f07488
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/join_reorder3-5-ade68a23d7b1a4f328623bb5a0f07488
diff --git a/sql/hive/src/test/resources/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/join_reorder3-6-8eb53fb8f05a43ee377aa1c927857e7c
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc
rename to sql/hive/src/test/resources/golden/join_reorder3-6-8eb53fb8f05a43ee377aa1c927857e7c
diff --git a/sql/hive/src/test/resources/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec b/sql/hive/src/test/resources/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/join_reorder3-7-5f4cfbbe53c5e808ee08b26514272034
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a
rename to sql/hive/src/test/resources/golden/join_reorder3-7-5f4cfbbe53c5e808ee08b26514272034
diff --git a/sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 b/sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13
index 06a83e95d4f79d31231238445463f21d587de8bb..33e6a1546ca65df70809a95a187180f11f2b3343 100644
--- a/sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13
+++ b/sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13
@@ -1 +1 @@
-2	12	2	22	2	12	2	12
\ No newline at end of file
+2	12	2	22	2	12	2	12
diff --git a/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f b/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f
index 872146532307a24150bfadd1bf63f5e2a41e14ed..19304c010452edd4780d6b69bb78cf1064888df4 100644
--- a/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f
+++ b/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f
@@ -1 +1 @@
-2	12	2	22	2	12
\ No newline at end of file
+2	12	2	22	2	12
diff --git a/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 b/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1
index 872146532307a24150bfadd1bf63f5e2a41e14ed..19304c010452edd4780d6b69bb78cf1064888df4 100644
--- a/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1
+++ b/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1
@@ -1 +1 @@
-2	12	2	22	2	12
\ No newline at end of file
+2	12	2	22	2	12
diff --git a/sql/hive/src/test/resources/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/join_reorder4-3-c0f14def6a135cc50cba364e810ce28e
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394
rename to sql/hive/src/test/resources/golden/join_reorder4-3-c0f14def6a135cc50cba364e810ce28e
diff --git a/sql/hive/src/test/resources/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/join_reorder4-4-ade68a23d7b1a4f328623bb5a0f07488
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1
rename to sql/hive/src/test/resources/golden/join_reorder4-4-ade68a23d7b1a4f328623bb5a0f07488
diff --git a/sql/hive/src/test/resources/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 b/sql/hive/src/test/resources/golden/join_reorder4-5-8eb53fb8f05a43ee377aa1c927857e7c
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29
rename to sql/hive/src/test/resources/golden/join_reorder4-5-8eb53fb8f05a43ee377aa1c927857e7c
diff --git a/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 b/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754
index 872146532307a24150bfadd1bf63f5e2a41e14ed..19304c010452edd4780d6b69bb78cf1064888df4 100644
--- a/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754
+++ b/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754
@@ -1 +1 @@
-2	12	2	22	2	12
\ No newline at end of file
+2	12	2	22	2	12
diff --git a/sql/hive/src/test/resources/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e b/sql/hive/src/test/resources/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 b/sql/hive/src/test/resources/golden/join_star-10-a9e579038e3d4826fdae475d7058ab82
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3
rename to sql/hive/src/test/resources/golden/join_star-10-a9e579038e3d4826fdae475d7058ab82
diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 b/sql/hive/src/test/resources/golden/join_star-11-72730ecdad9c0fd4c6ce64a0cb89fb74
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62
rename to sql/hive/src/test/resources/golden/join_star-11-72730ecdad9c0fd4c6ce64a0cb89fb74
diff --git a/sql/hive/src/test/resources/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae b/sql/hive/src/test/resources/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 b/sql/hive/src/test/resources/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 b/sql/hive/src/test/resources/golden/join_star-12-f581d6d305d652cd0f4e4fa912eb578d
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85
rename to sql/hive/src/test/resources/golden/join_star-12-f581d6d305d652cd0f4e4fa912eb578d
diff --git a/sql/hive/src/test/resources/golden/join_star-13-342b7249c9ce1484869169b1b33191cb b/sql/hive/src/test/resources/golden/join_star-13-342b7249c9ce1484869169b1b33191cb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 b/sql/hive/src/test/resources/golden/join_star-13-7268564732cbb7489248f9d818f80c14
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08
rename to sql/hive/src/test/resources/golden/join_star-13-7268564732cbb7489248f9d818f80c14
diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 b/sql/hive/src/test/resources/golden/join_star-14-2ee0fcf000f8687fc8941bf212477e57
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60
rename to sql/hive/src/test/resources/golden/join_star-14-2ee0fcf000f8687fc8941bf212477e57
diff --git a/sql/hive/src/test/resources/golden/join_star-14-75513308d30b781fd2e06d81963c4363 b/sql/hive/src/test/resources/golden/join_star-14-75513308d30b781fd2e06d81963c4363
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 b/sql/hive/src/test/resources/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 b/sql/hive/src/test/resources/golden/join_star-15-43b0b3b5e40044f8dbaeef2c7fc9e3e9
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7
rename to sql/hive/src/test/resources/golden/join_star-15-43b0b3b5e40044f8dbaeef2c7fc9e3e9
diff --git a/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 b/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965
+++ b/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 b/sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339
index e60bca6269264c2b4cdc57a342e1d7c82e215ff8..f97ee71bc2333d368f2dde016cfa2669da9630cc 100644
--- a/sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339
+++ b/sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339
@@ -7,4 +7,4 @@
 61	62	3
 71	72	3
 81	82	3
-91	92	3
\ No newline at end of file
+91	92	3
diff --git a/sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b b/sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b
index e08d04997c56b3de9fbf0e29c602b9a6354dc50e..e56cb5c03eaa4c9f7051928816a168df2497e15e 100644
--- a/sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b
+++ b/sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b
@@ -7,4 +7,4 @@
 61	62	3	4
 71	72	3	4
 81	82	3	4
-91	92	3	4
\ No newline at end of file
+91	92	3	4
diff --git a/sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 b/sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49
index b89911131bd139f5c2a4dbca599f1c89c09c05e4..80924380b7d764a7a522b51d6630221c6497a797 100644
--- a/sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49
+++ b/sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49
@@ -7,4 +7,4 @@
 61	62	3	3
 71	72	3	3
 81	82	3	3
-91	92	3	3
\ No newline at end of file
+91	92	3	3
diff --git a/sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 b/sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330
index b89911131bd139f5c2a4dbca599f1c89c09c05e4..80924380b7d764a7a522b51d6630221c6497a797 100644
--- a/sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330
+++ b/sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330
@@ -7,4 +7,4 @@
 61	62	3	3
 71	72	3	3
 81	82	3	3
-91	92	3	3
\ No newline at end of file
+91	92	3	3
diff --git a/sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 b/sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9
index 19611e75e33c3a3fff5d2895073da7ca987580f7..ebfbb5cdae3316efff6b605306b5641c4ae381aa 100644
--- a/sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9
+++ b/sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9
@@ -7,4 +7,4 @@
 61	62	3	3	4	4	4	4	4
 71	72	3	3	4	4	4	4	4
 81	82	3	3	4	4	4	4	4
-91	92	3	3	4	4	4	4	4
\ No newline at end of file
+91	92	3	3	4	4	4	4	4
diff --git a/sql/hive/src/test/resources/golden/join_star-8-a957982d8981ff0a35397ca449297024 b/sql/hive/src/test/resources/golden/join_star-8-a957982d8981ff0a35397ca449297024
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/join_star-8-c3d53a4daab9614a09870dc8e9571f74
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/join_star-8-c3d53a4daab9614a09870dc8e9571f74
diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/join_star-9-3f7ccccc2488de5f33a38cb3cc3eb628
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc
rename to sql/hive/src/test/resources/golden/join_star-9-3f7ccccc2488de5f33a38cb3cc3eb628
diff --git a/sql/hive/src/test/resources/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 b/sql/hive/src/test/resources/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 b/sql/hive/src/test/resources/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41
deleted file mode 100644
index 07433297e6de17ce0bf00bc35dff59d69762c74b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41
+++ /dev/null
@@ -1,6 +0,0 @@
-aint                	int                 	from deserializer   
-astring             	string              	from deserializer   
-lint                	array<int>          	from deserializer   
-lstring             	array<string>       	from deserializer   
-lintstring          	array<struct<myint:int,mystring:string,underscore_int:int>>	from deserializer   
-mstringstring       	map<string,string>  	from deserializer   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db b/sql/hive/src/test/resources/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b b/sql/hive/src/test/resources/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b
deleted file mode 100644
index 4aeaf4ff1f1b05061d4042e0abe643931c9d0e9d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b
+++ /dev/null
@@ -1,11 +0,0 @@
-1712634731	[{"myint":0,"mystring":"0","underscore_int":0}]
-465985200	[{"myint":1,"mystring":"1","underscore_int":1}]
--751827638	[{"myint":4,"mystring":"8","underscore_int":2}]
-477111222	[{"myint":9,"mystring":"27","underscore_int":3}]
--734328909	[{"myint":16,"mystring":"64","underscore_int":4}]
--1952710710	[{"myint":25,"mystring":"125","underscore_int":5}]
-1244525190	[{"myint":36,"mystring":"216","underscore_int":6}]
--1461153973	[{"myint":49,"mystring":"343","underscore_int":7}]
-1638581578	[{"myint":64,"mystring":"512","underscore_int":8}]
-336964413	[{"myint":81,"mystring":"729","underscore_int":9}]
-0	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 b/sql/hive/src/test/resources/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 b/sql/hive/src/test/resources/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 b/sql/hive/src/test/resources/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 b/sql/hive/src/test/resources/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b b/sql/hive/src/test/resources/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 b/sql/hive/src/test/resources/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef b/sql/hive/src/test/resources/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d b/sql/hive/src/test/resources/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 b/sql/hive/src/test/resources/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 b/sql/hive/src/test/resources/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 b/sql/hive/src/test/resources/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 b/sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42
deleted file mode 100644
index 0da0d93886e0196e679471cdce5f652134176d12..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42
+++ /dev/null
@@ -1,2 +0,0 @@
-key1	100
-key2	200
diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a b/sql/hive/src/test/resources/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 b/sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56
deleted file mode 100644
index 0da0d93886e0196e679471cdce5f652134176d12..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56
+++ /dev/null
@@ -1,2 +0,0 @@
-key1	100
-key2	200
diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe b/sql/hive/src/test/resources/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 b/sql/hive/src/test/resources/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 b/sql/hive/src/test/resources/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa b/sql/hive/src/test/resources/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa
deleted file mode 100644
index a2a7fdd233a2a69078b87a9ede1c806d72189790..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa
+++ /dev/null
@@ -1,10 +0,0 @@
-238	val_238	NULL
-86	val_86	NULL
-311	val_311	NULL
-27	val_27	NULL
-165	val_165	NULL
-409	val_409	NULL
-255	val_255	NULL
-278	val_278	NULL
-98	val_98	NULL
-484	val_484	NULL
diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 b/sql/hive/src/test/resources/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d b/sql/hive/src/test/resources/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d
deleted file mode 100644
index 37d918a386d7d804dba3297d7bc10b0ba4b19efa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d
+++ /dev/null
@@ -1,10 +0,0 @@
-238	val_238	4
-238	val_238	5
-86	val_86	4
-86	val_86	5
-311	val_311	4
-311	val_311	5
-27	val_27	4
-27	val_27	5
-165	val_165	4
-165	val_165	5
diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e b/sql/hive/src/test/resources/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 b/sql/hive/src/test/resources/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 b/sql/hive/src/test/resources/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120
deleted file mode 100644
index 3ad56f646ed8565d4244d30dc1f38e7e45dc237d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120
+++ /dev/null
@@ -1,10 +0,0 @@
-238	NULL	NULL
-86	NULL	NULL
-311	["val_311","val_311"]	val_311
-311	["val_311","val_311"]	val_311
-27	NULL	NULL
-165	NULL	NULL
-409	["val_409","val_409"]	val_409
-409	["val_409","val_409"]	val_409
-255	NULL	NULL
-278	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/lateral_view_ppd-10-a537ad7282d1c9957cdae74ad87c790b b/sql/hive/src/test/resources/golden/lateral_view_ppd-10-a537ad7282d1c9957cdae74ad87c790b
new file mode 100644
index 0000000000000000000000000000000000000000..cbeb8081f06c668ccfac55b178affb791d726b31
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/lateral_view_ppd-10-a537ad7282d1c9957cdae74ad87c790b
@@ -0,0 +1,6 @@
+val_0	2
+val_0	3
+val_0	2
+val_0	3
+val_0	2
+val_0	3
diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/lateral_view_ppd-9-dc6fea663d875b082d38bd326d21cd95
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a
rename to sql/hive/src/test/resources/golden/lateral_view_ppd-9-dc6fea663d875b082d38bd326d21cd95
diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-5-9c307c0559d735960ce77efa95b2b17b b/sql/hive/src/test/resources/golden/leftsemijoin-5-9c307c0559d735960ce77efa95b2b17b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/leftsemijoin-5-aba449db0d4fe6dc9771426e102bb543
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394
rename to sql/hive/src/test/resources/golden/leftsemijoin-5-aba449db0d4fe6dc9771426e102bb543
diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-6-82921fc96eef547ec0f71027ee88298c b/sql/hive/src/test/resources/golden/leftsemijoin-6-82921fc96eef547ec0f71027ee88298c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/leftsemijoin-6-9f50dce576b019c0be997055b8876621
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1
rename to sql/hive/src/test/resources/golden/leftsemijoin-6-9f50dce576b019c0be997055b8876621
diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-7-b30aa3b4a45db6b64bb46b4d9bd32ff0 b/sql/hive/src/test/resources/golden/leftsemijoin-7-b30aa3b4a45db6b64bb46b4d9bd32ff0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/leftsemijoin-7-fff6ca40e6048d52dc2d3afc68e8353e
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/leftsemijoin-7-fff6ca40e6048d52dc2d3afc68e8353e
diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/leftsemijoin_mr-1-5b2e555868faa404ea09928936178181
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc
rename to sql/hive/src/test/resources/golden/leftsemijoin_mr-1-5b2e555868faa404ea09928936178181
diff --git a/sql/hive/src/test/resources/golden/leftsemijoin_mr-1-aa3f07f028027ffd13ab5535dc821593 b/sql/hive/src/test/resources/golden/leftsemijoin_mr-1-aa3f07f028027ffd13ab5535dc821593
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/leftsemijoin_mr-3-645cf8b871c9b27418d6fa1d1bda9a52 b/sql/hive/src/test/resources/golden/leftsemijoin_mr-3-645cf8b871c9b27418d6fa1d1bda9a52
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/leftsemijoin_mr-3-c019cb2a855138da0d0b1e5c67cd6354
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a
rename to sql/hive/src/test/resources/golden/leftsemijoin_mr-3-c019cb2a855138da0d0b1e5c67cd6354
diff --git a/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 b/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02
index bf0d87ab1b2b0ec1a11a3973d2845b42413d9767..b8626c4cff2849624fb67f87cd0ad72b163671ad 100644
--- a/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02
+++ b/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02
@@ -1 +1 @@
-4
\ No newline at end of file
+4
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb b/sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 b/sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 b/sql/hive/src/test/resources/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f b/sql/hive/src/test/resources/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f
deleted file mode 100644
index 9e3b31ad52c139e0cbb56e5fcb76b57530768807..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f
+++ /dev/null
@@ -1,20 +0,0 @@
-0
-2
-4
-5
-8
-9
-10
-11
-12
-15
-17
-18
-19
-20
-24
-26
-27
-28
-30
-33
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 b/sql/hive/src/test/resources/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 b/sql/hive/src/test/resources/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78
deleted file mode 100644
index ebf6c0424c26db3b7ca40537ebbe0371e0f0e382..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78
+++ /dev/null
@@ -1,20 +0,0 @@
-0	1
-2	1
-4	1
-5	1
-8	1
-9	1
-10	1
-11	1
-12	1
-15	1
-17	1
-18	1
-19	1
-20	1
-24	1
-26	1
-27	1
-28	1
-30	1
-33	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 b/sql/hive/src/test/resources/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 b/sql/hive/src/test/resources/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 b/sql/hive/src/test/resources/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 b/sql/hive/src/test/resources/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60
deleted file mode 100644
index 153dcec21bc5bc5297c7ed6a6fa315f2ff4a0eec..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60
+++ /dev/null
@@ -1,20 +0,0 @@
-val_0	0
-val_2	2
-val_4	4
-val_8	8
-val_9	9
-val_10	10
-val_11	11
-val_5	15
-val_17	17
-val_19	19
-val_20	20
-val_12	24
-val_27	27
-val_28	28
-val_30	30
-val_15	30
-val_33	33
-val_34	34
-val_18	36
-val_41	41
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 b/sql/hive/src/test/resources/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 b/sql/hive/src/test/resources/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb b/sql/hive/src/test/resources/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 b/sql/hive/src/test/resources/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914
deleted file mode 100644
index ae8f0265b71cab2194010c436a49685913ddec67..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914
+++ /dev/null
@@ -1,20 +0,0 @@
-val_0	0
-val_10	10
-val_100	200
-val_103	206
-val_104	208
-val_105	105
-val_11	11
-val_111	111
-val_113	226
-val_114	114
-val_116	116
-val_118	236
-val_119	357
-val_12	24
-val_120	240
-val_125	250
-val_126	126
-val_128	384
-val_129	258
-val_131	131
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e b/sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a b/sql/hive/src/test/resources/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 b/sql/hive/src/test/resources/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976
deleted file mode 100644
index e604892422d594ebac13d7c461e90c70c38f44e2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976
+++ /dev/null
@@ -1,100 +0,0 @@
-0	val_0	val_0	val_0	val_0	val_0	val_0	val_0	val_0
-0	val_0	val_0	val_0	val_0	val_0	val_0	val_0	val_0
-0	val_0	val_0	val_0	val_0	val_0	val_0	val_0	val_0
-2	val_2	val_2	val_2	val_2	val_2	val_2	val_2	val_2
-4	val_4	val_4	val_4	val_4	val_4	val_4	val_4	val_4
-5	val_5	val_5	val_5	val_5	val_5	val_5	val_5	val_5
-5	val_5	val_5	val_5	val_5	val_5	val_5	val_5	val_5
-5	val_5	val_5	val_5	val_5	val_5	val_5	val_5	val_5
-8	val_8	val_8	val_8	val_8	val_8	val_8	val_8	val_8
-9	val_9	val_9	val_9	val_9	val_9	val_9	val_9	val_9
-10	val_10	val_10	val_10	val_10	val_10	val_10	val_10	val_10
-11	val_11	val_11	val_11	val_11	val_11	val_11	val_11	val_11
-12	val_12	val_12	val_12	val_12	val_12	val_12	val_12	val_12
-12	val_12	val_12	val_12	val_12	val_12	val_12	val_12	val_12
-15	val_15	val_15	val_15	val_15	val_15	val_15	val_15	val_15
-15	val_15	val_15	val_15	val_15	val_15	val_15	val_15	val_15
-17	val_17	val_17	val_17	val_17	val_17	val_17	val_17	val_17
-18	val_18	val_18	val_18	val_18	val_18	val_18	val_18	val_18
-18	val_18	val_18	val_18	val_18	val_18	val_18	val_18	val_18
-19	val_19	val_19	val_19	val_19	val_19	val_19	val_19	val_19
-20	val_20	val_20	val_20	val_20	val_20	val_20	val_20	val_20
-24	val_24	val_24	val_24	val_24	val_24	val_24	val_24	val_24
-24	val_24	val_24	val_24	val_24	val_24	val_24	val_24	val_24
-26	val_26	val_26	val_26	val_26	val_26	val_26	val_26	val_26
-26	val_26	val_26	val_26	val_26	val_26	val_26	val_26	val_26
-27	val_27	val_27	val_27	val_27	val_27	val_27	val_27	val_27
-28	val_28	val_28	val_28	val_28	val_28	val_28	val_28	val_28
-30	val_30	val_30	val_30	val_30	val_30	val_30	val_30	val_30
-33	val_33	val_33	val_33	val_33	val_33	val_33	val_33	val_33
-34	val_34	val_34	val_34	val_34	val_34	val_34	val_34	val_34
-35	val_35	val_35	val_35	val_35	val_35	val_35	val_35	val_35
-35	val_35	val_35	val_35	val_35	val_35	val_35	val_35	val_35
-35	val_35	val_35	val_35	val_35	val_35	val_35	val_35	val_35
-37	val_37	val_37	val_37	val_37	val_37	val_37	val_37	val_37
-37	val_37	val_37	val_37	val_37	val_37	val_37	val_37	val_37
-41	val_41	val_41	val_41	val_41	val_41	val_41	val_41	val_41
-42	val_42	val_42	val_42	val_42	val_42	val_42	val_42	val_42
-42	val_42	val_42	val_42	val_42	val_42	val_42	val_42	val_42
-43	val_43	val_43	val_43	val_43	val_43	val_43	val_43	val_43
-44	val_44	val_44	val_44	val_44	val_44	val_44	val_44	val_44
-47	val_47	val_47	val_47	val_47	val_47	val_47	val_47	val_47
-51	val_51	val_51	val_51	val_51	val_51	val_51	val_51	val_51
-51	val_51	val_51	val_51	val_51	val_51	val_51	val_51	val_51
-53	val_53	val_53	val_53	val_53	val_53	val_53	val_53	val_53
-54	val_54	val_54	val_54	val_54	val_54	val_54	val_54	val_54
-57	val_57	val_57	val_57	val_57	val_57	val_57	val_57	val_57
-58	val_58	val_58	val_58	val_58	val_58	val_58	val_58	val_58
-58	val_58	val_58	val_58	val_58	val_58	val_58	val_58	val_58
-64	val_64	val_64	val_64	val_64	val_64	val_64	val_64	val_64
-65	val_65	val_65	val_65	val_65	val_65	val_65	val_65	val_65
-66	val_66	val_66	val_66	val_66	val_66	val_66	val_66	val_66
-67	val_67	val_67	val_67	val_67	val_67	val_67	val_67	val_67
-67	val_67	val_67	val_67	val_67	val_67	val_67	val_67	val_67
-69	val_69	val_69	val_69	val_69	val_69	val_69	val_69	val_69
-70	val_70	val_70	val_70	val_70	val_70	val_70	val_70	val_70
-70	val_70	val_70	val_70	val_70	val_70	val_70	val_70	val_70
-70	val_70	val_70	val_70	val_70	val_70	val_70	val_70	val_70
-72	val_72	val_72	val_72	val_72	val_72	val_72	val_72	val_72
-72	val_72	val_72	val_72	val_72	val_72	val_72	val_72	val_72
-74	val_74	val_74	val_74	val_74	val_74	val_74	val_74	val_74
-76	val_76	val_76	val_76	val_76	val_76	val_76	val_76	val_76
-76	val_76	val_76	val_76	val_76	val_76	val_76	val_76	val_76
-77	val_77	val_77	val_77	val_77	val_77	val_77	val_77	val_77
-78	val_78	val_78	val_78	val_78	val_78	val_78	val_78	val_78
-80	val_80	val_80	val_80	val_80	val_80	val_80	val_80	val_80
-82	val_82	val_82	val_82	val_82	val_82	val_82	val_82	val_82
-83	val_83	val_83	val_83	val_83	val_83	val_83	val_83	val_83
-83	val_83	val_83	val_83	val_83	val_83	val_83	val_83	val_83
-84	val_84	val_84	val_84	val_84	val_84	val_84	val_84	val_84
-84	val_84	val_84	val_84	val_84	val_84	val_84	val_84	val_84
-85	val_85	val_85	val_85	val_85	val_85	val_85	val_85	val_85
-86	val_86	val_86	val_86	val_86	val_86	val_86	val_86	val_86
-87	val_87	val_87	val_87	val_87	val_87	val_87	val_87	val_87
-90	val_90	val_90	val_90	val_90	val_90	val_90	val_90	val_90
-90	val_90	val_90	val_90	val_90	val_90	val_90	val_90	val_90
-90	val_90	val_90	val_90	val_90	val_90	val_90	val_90	val_90
-92	val_92	val_92	val_92	val_92	val_92	val_92	val_92	val_92
-95	val_95	val_95	val_95	val_95	val_95	val_95	val_95	val_95
-95	val_95	val_95	val_95	val_95	val_95	val_95	val_95	val_95
-96	val_96	val_96	val_96	val_96	val_96	val_96	val_96	val_96
-97	val_97	val_97	val_97	val_97	val_97	val_97	val_97	val_97
-97	val_97	val_97	val_97	val_97	val_97	val_97	val_97	val_97
-98	val_98	val_98	val_98	val_98	val_98	val_98	val_98	val_98
-98	val_98	val_98	val_98	val_98	val_98	val_98	val_98	val_98
-100	val_100	val_100	val_100	val_100	val_100	val_100	val_100	val_100
-100	val_100	val_100	val_100	val_100	val_100	val_100	val_100	val_100
-103	val_103	val_103	val_103	val_103	val_103	val_103	val_103	val_103
-103	val_103	val_103	val_103	val_103	val_103	val_103	val_103	val_103
-104	val_104	val_104	val_104	val_104	val_104	val_104	val_104	val_104
-104	val_104	val_104	val_104	val_104	val_104	val_104	val_104	val_104
-105	val_105	val_105	val_105	val_105	val_105	val_105	val_105	val_105
-111	val_111	val_111	val_111	val_111	val_111	val_111	val_111	val_111
-113	val_113	val_113	val_113	val_113	val_113	val_113	val_113	val_113
-113	val_113	val_113	val_113	val_113	val_113	val_113	val_113	val_113
-114	val_114	val_114	val_114	val_114	val_114	val_114	val_114	val_114
-116	val_116	val_116	val_116	val_116	val_116	val_116	val_116	val_116
-118	val_118	val_118	val_118	val_118	val_118	val_118	val_118	val_118
-118	val_118	val_118	val_118	val_118	val_118	val_118	val_118	val_118
-119	val_119	val_119	val_119	val_119	val_119	val_119	val_119	val_119
-119	val_119	val_119	val_119	val_119	val_119	val_119	val_119	val_119
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a b/sql/hive/src/test/resources/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c b/sql/hive/src/test/resources/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c
deleted file mode 100644
index 92dc6ce9dbf9eaaf067047b83aaf421c2ec02564..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c
+++ /dev/null
@@ -1,100 +0,0 @@
-0
-200
-206
-208
-105
-10
-111
-226
-114
-116
-236
-357
-11
-240
-250
-126
-384
-258
-24
-131
-133
-268
-136
-274
-552
-143
-145
-292
-298
-150
-304
-153
-155
-156
-157
-158
-30
-160
-162
-163
-328
-330
-166
-501
-168
-676
-170
-344
-348
-350
-352
-177
-178
-358
-17
-180
-181
-183
-186
-561
-189
-36
-190
-382
-192
-579
-194
-390
-196
-394
-597
-19
-400
-201
-202
-406
-410
-414
-624
-418
-20
-426
-214
-432
-434
-218
-438
-442
-222
-446
-448
-226
-228
-458
-1150
-466
-235
-474
-476
-478
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 b/sql/hive/src/test/resources/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462
deleted file mode 100644
index 95f5492558a9bf6c389a1ca15c028fd77ab0a06c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462
+++ /dev/null
@@ -1,20 +0,0 @@
-0	val_0
-0	val_0
-0	val_0
-2	val_2
-4	val_4
-5	val_5
-5	val_5
-5	val_5
-8	val_8
-9	val_9
-10	val_10
-11	val_11
-12	val_12
-12	val_12
-15	val_15
-15	val_15
-17	val_17
-18	val_18
-18	val_18
-19	val_19
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f b/sql/hive/src/test/resources/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 b/sql/hive/src/test/resources/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7
deleted file mode 100644
index 0bac402cfa4977f40e8d3cac9c3eb4cdc5293a2d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7
+++ /dev/null
@@ -1,20 +0,0 @@
-498	val_498
-498	val_498
-498	val_498
-497	val_497
-496	val_496
-495	val_495
-494	val_494
-493	val_493
-492	val_492
-492	val_492
-491	val_491
-490	val_490
-489	val_489
-489	val_489
-489	val_489
-489	val_489
-487	val_487
-485	val_485
-484	val_484
-483	val_483
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 b/sql/hive/src/test/resources/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 b/sql/hive/src/test/resources/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921
deleted file mode 100644
index b57d0cc951566530dc5034ba08ec558e88dc9ba5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921
+++ /dev/null
@@ -1,20 +0,0 @@
-val_0	3
-val_10	11
-val_100	202
-val_103	208
-val_104	210
-val_105	106
-val_11	12
-val_111	112
-val_113	228
-val_114	115
-val_116	117
-val_118	238
-val_119	360
-val_12	26
-val_120	242
-val_125	252
-val_126	127
-val_128	387
-val_129	260
-val_131	132
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 b/sql/hive/src/test/resources/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a b/sql/hive/src/test/resources/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a
deleted file mode 100644
index 0e95c446f3b0dd452eb56d2ebafd0272e953c39d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a
+++ /dev/null
@@ -1,20 +0,0 @@
-val_0	1.0
-val_10	11.0
-val_100	101.0
-val_103	104.0
-val_104	105.0
-val_105	106.0
-val_11	12.0
-val_111	112.0
-val_113	114.0
-val_114	115.0
-val_116	117.0
-val_118	119.0
-val_119	120.0
-val_12	13.0
-val_120	121.0
-val_125	126.0
-val_126	127.0
-val_128	129.0
-val_129	130.0
-val_131	132.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-7-fb7bf3783d4fb43673a202c4111d9092 b/sql/hive/src/test/resources/golden/limit_pushdown_negative-7-fb7bf3783d4fb43673a202c4111d9092
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/literal_double-0-10ef1098e35d900983be3814de8f974f b/sql/hive/src/test/resources/golden/literal_double-0-10ef1098e35d900983be3814de8f974f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/literal_double-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/literal_double-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/literal_double-1-10ef1098e35d900983be3814de8f974f
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394
rename to sql/hive/src/test/resources/golden/literal_double-1-10ef1098e35d900983be3814de8f974f
diff --git a/sql/hive/src/test/resources/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 b/sql/hive/src/test/resources/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13
deleted file mode 100644
index 69505e73feb1d1a5ca99efc042d13c53f12bccd0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13
+++ /dev/null
@@ -1 +0,0 @@
-3.14	-3.14	3.14E8	3.14E-8	-3.14E8	-3.14E-8	3.14E8	3.14E8	3.14E-8
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/literal_double-2-3863c17e03c9c1cd68452106a8721d13 b/sql/hive/src/test/resources/golden/literal_double-2-3863c17e03c9c1cd68452106a8721d13
new file mode 100644
index 0000000000000000000000000000000000000000..24ca45210038a9aadd019820e4df470885fb2f65
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/literal_double-2-3863c17e03c9c1cd68452106a8721d13
@@ -0,0 +1 @@
+3.14	-3.14	3.14E8	3.14E-8	-3.14E8	-3.14E-8	3.14E8	3.14E8	3.14E-8
diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/literal_ints-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/literal_ints-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 b/sql/hive/src/test/resources/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/literal_ints-1-5ffd1b49cdda4149aef2c61c53a56890
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1
rename to sql/hive/src/test/resources/golden/literal_ints-1-5ffd1b49cdda4149aef2c61c53a56890
diff --git a/sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 b/sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4
deleted file mode 100644
index 6dc85dd550540f066c26ccb834da42e4c6596203..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4
+++ /dev/null
@@ -1 +0,0 @@
-100	100	100	100
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/literal_ints-2-b41c42ce2f6ba483b68bb08752b95ec4 b/sql/hive/src/test/resources/golden/literal_ints-2-b41c42ce2f6ba483b68bb08752b95ec4
new file mode 100644
index 0000000000000000000000000000000000000000..46a059f07aca66f64ada4091705a2d20cf8bc2f0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/literal_ints-2-b41c42ce2f6ba483b68bb08752b95ec4
@@ -0,0 +1 @@
+100	100	100	100
diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/literal_string-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/literal_string-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 b/sql/hive/src/test/resources/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d b/sql/hive/src/test/resources/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d
deleted file mode 100644
index 6323f4efa99c71a3c36d777b1d6e2b968e2c4527..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d
+++ /dev/null
@@ -1 +0,0 @@
-facebook	facebook	facebook	facebook	facebook	facebook	facebook	facebook	facebook	facebook
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/literal_string-1-9b48d41e5c9e41ddc070e2fd31ace15
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/literal_string-1-9b48d41e5c9e41ddc070e2fd31ace15
diff --git a/sql/hive/src/test/resources/golden/literal_string-2-2cf4b7268b47246afdf6c792acca379d b/sql/hive/src/test/resources/golden/literal_string-2-2cf4b7268b47246afdf6c792acca379d
new file mode 100644
index 0000000000000000000000000000000000000000..1d05317d625474406aab550b993eb75d0ce33f15
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/literal_string-2-2cf4b7268b47246afdf6c792acca379d
@@ -0,0 +1 @@
+facebook	facebook	facebook	facebook	facebook	facebook	facebook	facebook	facebook	facebook
diff --git a/sql/hive/src/test/resources/golden/load_binary_data-0-491edd0c42ceb79e799ba50555bc8c15 b/sql/hive/src/test/resources/golden/load_binary_data-0-491edd0c42ceb79e799ba50555bc8c15
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_binary_data-1-5d72f8449b69df3c08e3f444f09428bc b/sql/hive/src/test/resources/golden/load_binary_data-1-5d72f8449b69df3c08e3f444f09428bc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_binary_data-2-242b1655c7e7325ee9f26552ea8fc25 b/sql/hive/src/test/resources/golden/load_binary_data-2-242b1655c7e7325ee9f26552ea8fc25
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_binary_data-3-2a72df8d3e398d0963ef91162ce7d268 b/sql/hive/src/test/resources/golden/load_binary_data-3-2a72df8d3e398d0963ef91162ce7d268
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca b/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca
deleted file mode 100644
index de5212a3c320f772b6a3a2b246202cd99d07c012..0000000000000000000000000000000000000000
Binary files a/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca and /dev/null differ
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part1-10-c66fea7ad025cd1f513a98a4cc1036b1 b/sql/hive/src/test/resources/golden/load_dyn_part1-10-c66fea7ad025cd1f513a98a4cc1036b1
new file mode 100644
index 0000000000000000000000000000000000000000..04b36182974f919450daa1dacd7e2371b61bdb51
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/load_dyn_part1-10-c66fea7ad025cd1f513a98a4cc1036b1
@@ -0,0 +1,1000 @@
+0	val_0	2008-04-08	11
+0	val_0	2008-04-08	11
+0	val_0	2008-04-08	11
+2	val_2	2008-04-08	11
+4	val_4	2008-04-08	11
+5	val_5	2008-04-08	11
+5	val_5	2008-04-08	11
+5	val_5	2008-04-08	11
+8	val_8	2008-04-08	11
+9	val_9	2008-04-08	11
+10	val_10	2008-04-08	11
+11	val_11	2008-04-08	11
+12	val_12	2008-04-08	11
+12	val_12	2008-04-08	11
+15	val_15	2008-04-08	11
+15	val_15	2008-04-08	11
+17	val_17	2008-04-08	11
+18	val_18	2008-04-08	11
+18	val_18	2008-04-08	11
+19	val_19	2008-04-08	11
+20	val_20	2008-04-08	11
+24	val_24	2008-04-08	11
+24	val_24	2008-04-08	11
+26	val_26	2008-04-08	11
+26	val_26	2008-04-08	11
+27	val_27	2008-04-08	11
+28	val_28	2008-04-08	11
+30	val_30	2008-04-08	11
+33	val_33	2008-04-08	11
+34	val_34	2008-04-08	11
+35	val_35	2008-04-08	11
+35	val_35	2008-04-08	11
+35	val_35	2008-04-08	11
+37	val_37	2008-04-08	11
+37	val_37	2008-04-08	11
+41	val_41	2008-04-08	11
+42	val_42	2008-04-08	11
+42	val_42	2008-04-08	11
+43	val_43	2008-04-08	11
+44	val_44	2008-04-08	11
+47	val_47	2008-04-08	11
+51	val_51	2008-04-08	11
+51	val_51	2008-04-08	11
+53	val_53	2008-04-08	11
+54	val_54	2008-04-08	11
+57	val_57	2008-04-08	11
+58	val_58	2008-04-08	11
+58	val_58	2008-04-08	11
+64	val_64	2008-04-08	11
+65	val_65	2008-04-08	11
+66	val_66	2008-04-08	11
+67	val_67	2008-04-08	11
+67	val_67	2008-04-08	11
+69	val_69	2008-04-08	11
+70	val_70	2008-04-08	11
+70	val_70	2008-04-08	11
+70	val_70	2008-04-08	11
+72	val_72	2008-04-08	11
+72	val_72	2008-04-08	11
+74	val_74	2008-04-08	11
+76	val_76	2008-04-08	11
+76	val_76	2008-04-08	11
+77	val_77	2008-04-08	11
+78	val_78	2008-04-08	11
+80	val_80	2008-04-08	11
+82	val_82	2008-04-08	11
+83	val_83	2008-04-08	11
+83	val_83	2008-04-08	11
+84	val_84	2008-04-08	11
+84	val_84	2008-04-08	11
+85	val_85	2008-04-08	11
+86	val_86	2008-04-08	11
+87	val_87	2008-04-08	11
+90	val_90	2008-04-08	11
+90	val_90	2008-04-08	11
+90	val_90	2008-04-08	11
+92	val_92	2008-04-08	11
+95	val_95	2008-04-08	11
+95	val_95	2008-04-08	11
+96	val_96	2008-04-08	11
+97	val_97	2008-04-08	11
+97	val_97	2008-04-08	11
+98	val_98	2008-04-08	11
+98	val_98	2008-04-08	11
+100	val_100	2008-04-08	11
+100	val_100	2008-04-08	11
+103	val_103	2008-04-08	11
+103	val_103	2008-04-08	11
+104	val_104	2008-04-08	11
+104	val_104	2008-04-08	11
+105	val_105	2008-04-08	11
+111	val_111	2008-04-08	11
+113	val_113	2008-04-08	11
+113	val_113	2008-04-08	11
+114	val_114	2008-04-08	11
+116	val_116	2008-04-08	11
+118	val_118	2008-04-08	11
+118	val_118	2008-04-08	11
+119	val_119	2008-04-08	11
+119	val_119	2008-04-08	11
+119	val_119	2008-04-08	11
+120	val_120	2008-04-08	11
+120	val_120	2008-04-08	11
+125	val_125	2008-04-08	11
+125	val_125	2008-04-08	11
+126	val_126	2008-04-08	11
+128	val_128	2008-04-08	11
+128	val_128	2008-04-08	11
+128	val_128	2008-04-08	11
+129	val_129	2008-04-08	11
+129	val_129	2008-04-08	11
+131	val_131	2008-04-08	11
+133	val_133	2008-04-08	11
+134	val_134	2008-04-08	11
+134	val_134	2008-04-08	11
+136	val_136	2008-04-08	11
+137	val_137	2008-04-08	11
+137	val_137	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+143	val_143	2008-04-08	11
+145	val_145	2008-04-08	11
+146	val_146	2008-04-08	11
+146	val_146	2008-04-08	11
+149	val_149	2008-04-08	11
+149	val_149	2008-04-08	11
+150	val_150	2008-04-08	11
+152	val_152	2008-04-08	11
+152	val_152	2008-04-08	11
+153	val_153	2008-04-08	11
+155	val_155	2008-04-08	11
+156	val_156	2008-04-08	11
+157	val_157	2008-04-08	11
+158	val_158	2008-04-08	11
+160	val_160	2008-04-08	11
+162	val_162	2008-04-08	11
+163	val_163	2008-04-08	11
+164	val_164	2008-04-08	11
+164	val_164	2008-04-08	11
+165	val_165	2008-04-08	11
+165	val_165	2008-04-08	11
+166	val_166	2008-04-08	11
+167	val_167	2008-04-08	11
+167	val_167	2008-04-08	11
+167	val_167	2008-04-08	11
+168	val_168	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+170	val_170	2008-04-08	11
+172	val_172	2008-04-08	11
+172	val_172	2008-04-08	11
+174	val_174	2008-04-08	11
+174	val_174	2008-04-08	11
+175	val_175	2008-04-08	11
+175	val_175	2008-04-08	11
+176	val_176	2008-04-08	11
+176	val_176	2008-04-08	11
+177	val_177	2008-04-08	11
+178	val_178	2008-04-08	11
+179	val_179	2008-04-08	11
+179	val_179	2008-04-08	11
+180	val_180	2008-04-08	11
+181	val_181	2008-04-08	11
+183	val_183	2008-04-08	11
+186	val_186	2008-04-08	11
+187	val_187	2008-04-08	11
+187	val_187	2008-04-08	11
+187	val_187	2008-04-08	11
+189	val_189	2008-04-08	11
+190	val_190	2008-04-08	11
+191	val_191	2008-04-08	11
+191	val_191	2008-04-08	11
+192	val_192	2008-04-08	11
+193	val_193	2008-04-08	11
+193	val_193	2008-04-08	11
+193	val_193	2008-04-08	11
+194	val_194	2008-04-08	11
+195	val_195	2008-04-08	11
+195	val_195	2008-04-08	11
+196	val_196	2008-04-08	11
+197	val_197	2008-04-08	11
+197	val_197	2008-04-08	11
+199	val_199	2008-04-08	11
+199	val_199	2008-04-08	11
+199	val_199	2008-04-08	11
+200	val_200	2008-04-08	11
+200	val_200	2008-04-08	11
+201	val_201	2008-04-08	11
+202	val_202	2008-04-08	11
+203	val_203	2008-04-08	11
+203	val_203	2008-04-08	11
+205	val_205	2008-04-08	11
+205	val_205	2008-04-08	11
+207	val_207	2008-04-08	11
+207	val_207	2008-04-08	11
+208	val_208	2008-04-08	11
+208	val_208	2008-04-08	11
+208	val_208	2008-04-08	11
+209	val_209	2008-04-08	11
+209	val_209	2008-04-08	11
+213	val_213	2008-04-08	11
+213	val_213	2008-04-08	11
+214	val_214	2008-04-08	11
+216	val_216	2008-04-08	11
+216	val_216	2008-04-08	11
+217	val_217	2008-04-08	11
+217	val_217	2008-04-08	11
+218	val_218	2008-04-08	11
+219	val_219	2008-04-08	11
+219	val_219	2008-04-08	11
+221	val_221	2008-04-08	11
+221	val_221	2008-04-08	11
+222	val_222	2008-04-08	11
+223	val_223	2008-04-08	11
+223	val_223	2008-04-08	11
+224	val_224	2008-04-08	11
+224	val_224	2008-04-08	11
+226	val_226	2008-04-08	11
+228	val_228	2008-04-08	11
+229	val_229	2008-04-08	11
+229	val_229	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+233	val_233	2008-04-08	11
+233	val_233	2008-04-08	11
+235	val_235	2008-04-08	11
+237	val_237	2008-04-08	11
+237	val_237	2008-04-08	11
+238	val_238	2008-04-08	11
+238	val_238	2008-04-08	11
+239	val_239	2008-04-08	11
+239	val_239	2008-04-08	11
+241	val_241	2008-04-08	11
+242	val_242	2008-04-08	11
+242	val_242	2008-04-08	11
+244	val_244	2008-04-08	11
+247	val_247	2008-04-08	11
+248	val_248	2008-04-08	11
+249	val_249	2008-04-08	11
+252	val_252	2008-04-08	11
+255	val_255	2008-04-08	11
+255	val_255	2008-04-08	11
+256	val_256	2008-04-08	11
+256	val_256	2008-04-08	11
+257	val_257	2008-04-08	11
+258	val_258	2008-04-08	11
+260	val_260	2008-04-08	11
+262	val_262	2008-04-08	11
+263	val_263	2008-04-08	11
+265	val_265	2008-04-08	11
+265	val_265	2008-04-08	11
+266	val_266	2008-04-08	11
+272	val_272	2008-04-08	11
+272	val_272	2008-04-08	11
+273	val_273	2008-04-08	11
+273	val_273	2008-04-08	11
+273	val_273	2008-04-08	11
+274	val_274	2008-04-08	11
+275	val_275	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+278	val_278	2008-04-08	11
+278	val_278	2008-04-08	11
+280	val_280	2008-04-08	11
+280	val_280	2008-04-08	11
+281	val_281	2008-04-08	11
+281	val_281	2008-04-08	11
+282	val_282	2008-04-08	11
+282	val_282	2008-04-08	11
+283	val_283	2008-04-08	11
+284	val_284	2008-04-08	11
+285	val_285	2008-04-08	11
+286	val_286	2008-04-08	11
+287	val_287	2008-04-08	11
+288	val_288	2008-04-08	11
+288	val_288	2008-04-08	11
+289	val_289	2008-04-08	11
+291	val_291	2008-04-08	11
+292	val_292	2008-04-08	11
+296	val_296	2008-04-08	11
+298	val_298	2008-04-08	11
+298	val_298	2008-04-08	11
+298	val_298	2008-04-08	11
+302	val_302	2008-04-08	11
+305	val_305	2008-04-08	11
+306	val_306	2008-04-08	11
+307	val_307	2008-04-08	11
+307	val_307	2008-04-08	11
+308	val_308	2008-04-08	11
+309	val_309	2008-04-08	11
+309	val_309	2008-04-08	11
+310	val_310	2008-04-08	11
+311	val_311	2008-04-08	11
+311	val_311	2008-04-08	11
+311	val_311	2008-04-08	11
+315	val_315	2008-04-08	11
+316	val_316	2008-04-08	11
+316	val_316	2008-04-08	11
+316	val_316	2008-04-08	11
+317	val_317	2008-04-08	11
+317	val_317	2008-04-08	11
+318	val_318	2008-04-08	11
+318	val_318	2008-04-08	11
+318	val_318	2008-04-08	11
+321	val_321	2008-04-08	11
+321	val_321	2008-04-08	11
+322	val_322	2008-04-08	11
+322	val_322	2008-04-08	11
+323	val_323	2008-04-08	11
+325	val_325	2008-04-08	11
+325	val_325	2008-04-08	11
+327	val_327	2008-04-08	11
+327	val_327	2008-04-08	11
+327	val_327	2008-04-08	11
+331	val_331	2008-04-08	11
+331	val_331	2008-04-08	11
+332	val_332	2008-04-08	11
+333	val_333	2008-04-08	11
+333	val_333	2008-04-08	11
+335	val_335	2008-04-08	11
+336	val_336	2008-04-08	11
+338	val_338	2008-04-08	11
+339	val_339	2008-04-08	11
+341	val_341	2008-04-08	11
+342	val_342	2008-04-08	11
+342	val_342	2008-04-08	11
+344	val_344	2008-04-08	11
+344	val_344	2008-04-08	11
+345	val_345	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+351	val_351	2008-04-08	11
+353	val_353	2008-04-08	11
+353	val_353	2008-04-08	11
+356	val_356	2008-04-08	11
+360	val_360	2008-04-08	11
+362	val_362	2008-04-08	11
+364	val_364	2008-04-08	11
+365	val_365	2008-04-08	11
+366	val_366	2008-04-08	11
+367	val_367	2008-04-08	11
+367	val_367	2008-04-08	11
+368	val_368	2008-04-08	11
+369	val_369	2008-04-08	11
+369	val_369	2008-04-08	11
+369	val_369	2008-04-08	11
+373	val_373	2008-04-08	11
+374	val_374	2008-04-08	11
+375	val_375	2008-04-08	11
+377	val_377	2008-04-08	11
+378	val_378	2008-04-08	11
+379	val_379	2008-04-08	11
+382	val_382	2008-04-08	11
+382	val_382	2008-04-08	11
+384	val_384	2008-04-08	11
+384	val_384	2008-04-08	11
+384	val_384	2008-04-08	11
+386	val_386	2008-04-08	11
+389	val_389	2008-04-08	11
+392	val_392	2008-04-08	11
+393	val_393	2008-04-08	11
+394	val_394	2008-04-08	11
+395	val_395	2008-04-08	11
+395	val_395	2008-04-08	11
+396	val_396	2008-04-08	11
+396	val_396	2008-04-08	11
+396	val_396	2008-04-08	11
+397	val_397	2008-04-08	11
+397	val_397	2008-04-08	11
+399	val_399	2008-04-08	11
+399	val_399	2008-04-08	11
+400	val_400	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+402	val_402	2008-04-08	11
+403	val_403	2008-04-08	11
+403	val_403	2008-04-08	11
+403	val_403	2008-04-08	11
+404	val_404	2008-04-08	11
+404	val_404	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+407	val_407	2008-04-08	11
+409	val_409	2008-04-08	11
+409	val_409	2008-04-08	11
+409	val_409	2008-04-08	11
+411	val_411	2008-04-08	11
+413	val_413	2008-04-08	11
+413	val_413	2008-04-08	11
+414	val_414	2008-04-08	11
+414	val_414	2008-04-08	11
+417	val_417	2008-04-08	11
+417	val_417	2008-04-08	11
+417	val_417	2008-04-08	11
+418	val_418	2008-04-08	11
+419	val_419	2008-04-08	11
+421	val_421	2008-04-08	11
+424	val_424	2008-04-08	11
+424	val_424	2008-04-08	11
+427	val_427	2008-04-08	11
+429	val_429	2008-04-08	11
+429	val_429	2008-04-08	11
+430	val_430	2008-04-08	11
+430	val_430	2008-04-08	11
+430	val_430	2008-04-08	11
+431	val_431	2008-04-08	11
+431	val_431	2008-04-08	11
+431	val_431	2008-04-08	11
+432	val_432	2008-04-08	11
+435	val_435	2008-04-08	11
+436	val_436	2008-04-08	11
+437	val_437	2008-04-08	11
+438	val_438	2008-04-08	11
+438	val_438	2008-04-08	11
+438	val_438	2008-04-08	11
+439	val_439	2008-04-08	11
+439	val_439	2008-04-08	11
+443	val_443	2008-04-08	11
+444	val_444	2008-04-08	11
+446	val_446	2008-04-08	11
+448	val_448	2008-04-08	11
+449	val_449	2008-04-08	11
+452	val_452	2008-04-08	11
+453	val_453	2008-04-08	11
+454	val_454	2008-04-08	11
+454	val_454	2008-04-08	11
+454	val_454	2008-04-08	11
+455	val_455	2008-04-08	11
+457	val_457	2008-04-08	11
+458	val_458	2008-04-08	11
+458	val_458	2008-04-08	11
+459	val_459	2008-04-08	11
+459	val_459	2008-04-08	11
+460	val_460	2008-04-08	11
+462	val_462	2008-04-08	11
+462	val_462	2008-04-08	11
+463	val_463	2008-04-08	11
+463	val_463	2008-04-08	11
+466	val_466	2008-04-08	11
+466	val_466	2008-04-08	11
+466	val_466	2008-04-08	11
+467	val_467	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+470	val_470	2008-04-08	11
+472	val_472	2008-04-08	11
+475	val_475	2008-04-08	11
+477	val_477	2008-04-08	11
+478	val_478	2008-04-08	11
+478	val_478	2008-04-08	11
+479	val_479	2008-04-08	11
+480	val_480	2008-04-08	11
+480	val_480	2008-04-08	11
+480	val_480	2008-04-08	11
+481	val_481	2008-04-08	11
+482	val_482	2008-04-08	11
+483	val_483	2008-04-08	11
+484	val_484	2008-04-08	11
+485	val_485	2008-04-08	11
+487	val_487	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+490	val_490	2008-04-08	11
+491	val_491	2008-04-08	11
+492	val_492	2008-04-08	11
+492	val_492	2008-04-08	11
+493	val_493	2008-04-08	11
+494	val_494	2008-04-08	11
+495	val_495	2008-04-08	11
+496	val_496	2008-04-08	11
+497	val_497	2008-04-08	11
+498	val_498	2008-04-08	11
+498	val_498	2008-04-08	11
+498	val_498	2008-04-08	11
+0	val_0	2008-04-08	12
+0	val_0	2008-04-08	12
+0	val_0	2008-04-08	12
+2	val_2	2008-04-08	12
+4	val_4	2008-04-08	12
+5	val_5	2008-04-08	12
+5	val_5	2008-04-08	12
+5	val_5	2008-04-08	12
+8	val_8	2008-04-08	12
+9	val_9	2008-04-08	12
+10	val_10	2008-04-08	12
+11	val_11	2008-04-08	12
+12	val_12	2008-04-08	12
+12	val_12	2008-04-08	12
+15	val_15	2008-04-08	12
+15	val_15	2008-04-08	12
+17	val_17	2008-04-08	12
+18	val_18	2008-04-08	12
+18	val_18	2008-04-08	12
+19	val_19	2008-04-08	12
+20	val_20	2008-04-08	12
+24	val_24	2008-04-08	12
+24	val_24	2008-04-08	12
+26	val_26	2008-04-08	12
+26	val_26	2008-04-08	12
+27	val_27	2008-04-08	12
+28	val_28	2008-04-08	12
+30	val_30	2008-04-08	12
+33	val_33	2008-04-08	12
+34	val_34	2008-04-08	12
+35	val_35	2008-04-08	12
+35	val_35	2008-04-08	12
+35	val_35	2008-04-08	12
+37	val_37	2008-04-08	12
+37	val_37	2008-04-08	12
+41	val_41	2008-04-08	12
+42	val_42	2008-04-08	12
+42	val_42	2008-04-08	12
+43	val_43	2008-04-08	12
+44	val_44	2008-04-08	12
+47	val_47	2008-04-08	12
+51	val_51	2008-04-08	12
+51	val_51	2008-04-08	12
+53	val_53	2008-04-08	12
+54	val_54	2008-04-08	12
+57	val_57	2008-04-08	12
+58	val_58	2008-04-08	12
+58	val_58	2008-04-08	12
+64	val_64	2008-04-08	12
+65	val_65	2008-04-08	12
+66	val_66	2008-04-08	12
+67	val_67	2008-04-08	12
+67	val_67	2008-04-08	12
+69	val_69	2008-04-08	12
+70	val_70	2008-04-08	12
+70	val_70	2008-04-08	12
+70	val_70	2008-04-08	12
+72	val_72	2008-04-08	12
+72	val_72	2008-04-08	12
+74	val_74	2008-04-08	12
+76	val_76	2008-04-08	12
+76	val_76	2008-04-08	12
+77	val_77	2008-04-08	12
+78	val_78	2008-04-08	12
+80	val_80	2008-04-08	12
+82	val_82	2008-04-08	12
+83	val_83	2008-04-08	12
+83	val_83	2008-04-08	12
+84	val_84	2008-04-08	12
+84	val_84	2008-04-08	12
+85	val_85	2008-04-08	12
+86	val_86	2008-04-08	12
+87	val_87	2008-04-08	12
+90	val_90	2008-04-08	12
+90	val_90	2008-04-08	12
+90	val_90	2008-04-08	12
+92	val_92	2008-04-08	12
+95	val_95	2008-04-08	12
+95	val_95	2008-04-08	12
+96	val_96	2008-04-08	12
+97	val_97	2008-04-08	12
+97	val_97	2008-04-08	12
+98	val_98	2008-04-08	12
+98	val_98	2008-04-08	12
+100	val_100	2008-04-08	12
+100	val_100	2008-04-08	12
+103	val_103	2008-04-08	12
+103	val_103	2008-04-08	12
+104	val_104	2008-04-08	12
+104	val_104	2008-04-08	12
+105	val_105	2008-04-08	12
+111	val_111	2008-04-08	12
+113	val_113	2008-04-08	12
+113	val_113	2008-04-08	12
+114	val_114	2008-04-08	12
+116	val_116	2008-04-08	12
+118	val_118	2008-04-08	12
+118	val_118	2008-04-08	12
+119	val_119	2008-04-08	12
+119	val_119	2008-04-08	12
+119	val_119	2008-04-08	12
+120	val_120	2008-04-08	12
+120	val_120	2008-04-08	12
+125	val_125	2008-04-08	12
+125	val_125	2008-04-08	12
+126	val_126	2008-04-08	12
+128	val_128	2008-04-08	12
+128	val_128	2008-04-08	12
+128	val_128	2008-04-08	12
+129	val_129	2008-04-08	12
+129	val_129	2008-04-08	12
+131	val_131	2008-04-08	12
+133	val_133	2008-04-08	12
+134	val_134	2008-04-08	12
+134	val_134	2008-04-08	12
+136	val_136	2008-04-08	12
+137	val_137	2008-04-08	12
+137	val_137	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+143	val_143	2008-04-08	12
+145	val_145	2008-04-08	12
+146	val_146	2008-04-08	12
+146	val_146	2008-04-08	12
+149	val_149	2008-04-08	12
+149	val_149	2008-04-08	12
+150	val_150	2008-04-08	12
+152	val_152	2008-04-08	12
+152	val_152	2008-04-08	12
+153	val_153	2008-04-08	12
+155	val_155	2008-04-08	12
+156	val_156	2008-04-08	12
+157	val_157	2008-04-08	12
+158	val_158	2008-04-08	12
+160	val_160	2008-04-08	12
+162	val_162	2008-04-08	12
+163	val_163	2008-04-08	12
+164	val_164	2008-04-08	12
+164	val_164	2008-04-08	12
+165	val_165	2008-04-08	12
+165	val_165	2008-04-08	12
+166	val_166	2008-04-08	12
+167	val_167	2008-04-08	12
+167	val_167	2008-04-08	12
+167	val_167	2008-04-08	12
+168	val_168	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+170	val_170	2008-04-08	12
+172	val_172	2008-04-08	12
+172	val_172	2008-04-08	12
+174	val_174	2008-04-08	12
+174	val_174	2008-04-08	12
+175	val_175	2008-04-08	12
+175	val_175	2008-04-08	12
+176	val_176	2008-04-08	12
+176	val_176	2008-04-08	12
+177	val_177	2008-04-08	12
+178	val_178	2008-04-08	12
+179	val_179	2008-04-08	12
+179	val_179	2008-04-08	12
+180	val_180	2008-04-08	12
+181	val_181	2008-04-08	12
+183	val_183	2008-04-08	12
+186	val_186	2008-04-08	12
+187	val_187	2008-04-08	12
+187	val_187	2008-04-08	12
+187	val_187	2008-04-08	12
+189	val_189	2008-04-08	12
+190	val_190	2008-04-08	12
+191	val_191	2008-04-08	12
+191	val_191	2008-04-08	12
+192	val_192	2008-04-08	12
+193	val_193	2008-04-08	12
+193	val_193	2008-04-08	12
+193	val_193	2008-04-08	12
+194	val_194	2008-04-08	12
+195	val_195	2008-04-08	12
+195	val_195	2008-04-08	12
+196	val_196	2008-04-08	12
+197	val_197	2008-04-08	12
+197	val_197	2008-04-08	12
+199	val_199	2008-04-08	12
+199	val_199	2008-04-08	12
+199	val_199	2008-04-08	12
+200	val_200	2008-04-08	12
+200	val_200	2008-04-08	12
+201	val_201	2008-04-08	12
+202	val_202	2008-04-08	12
+203	val_203	2008-04-08	12
+203	val_203	2008-04-08	12
+205	val_205	2008-04-08	12
+205	val_205	2008-04-08	12
+207	val_207	2008-04-08	12
+207	val_207	2008-04-08	12
+208	val_208	2008-04-08	12
+208	val_208	2008-04-08	12
+208	val_208	2008-04-08	12
+209	val_209	2008-04-08	12
+209	val_209	2008-04-08	12
+213	val_213	2008-04-08	12
+213	val_213	2008-04-08	12
+214	val_214	2008-04-08	12
+216	val_216	2008-04-08	12
+216	val_216	2008-04-08	12
+217	val_217	2008-04-08	12
+217	val_217	2008-04-08	12
+218	val_218	2008-04-08	12
+219	val_219	2008-04-08	12
+219	val_219	2008-04-08	12
+221	val_221	2008-04-08	12
+221	val_221	2008-04-08	12
+222	val_222	2008-04-08	12
+223	val_223	2008-04-08	12
+223	val_223	2008-04-08	12
+224	val_224	2008-04-08	12
+224	val_224	2008-04-08	12
+226	val_226	2008-04-08	12
+228	val_228	2008-04-08	12
+229	val_229	2008-04-08	12
+229	val_229	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+233	val_233	2008-04-08	12
+233	val_233	2008-04-08	12
+235	val_235	2008-04-08	12
+237	val_237	2008-04-08	12
+237	val_237	2008-04-08	12
+238	val_238	2008-04-08	12
+238	val_238	2008-04-08	12
+239	val_239	2008-04-08	12
+239	val_239	2008-04-08	12
+241	val_241	2008-04-08	12
+242	val_242	2008-04-08	12
+242	val_242	2008-04-08	12
+244	val_244	2008-04-08	12
+247	val_247	2008-04-08	12
+248	val_248	2008-04-08	12
+249	val_249	2008-04-08	12
+252	val_252	2008-04-08	12
+255	val_255	2008-04-08	12
+255	val_255	2008-04-08	12
+256	val_256	2008-04-08	12
+256	val_256	2008-04-08	12
+257	val_257	2008-04-08	12
+258	val_258	2008-04-08	12
+260	val_260	2008-04-08	12
+262	val_262	2008-04-08	12
+263	val_263	2008-04-08	12
+265	val_265	2008-04-08	12
+265	val_265	2008-04-08	12
+266	val_266	2008-04-08	12
+272	val_272	2008-04-08	12
+272	val_272	2008-04-08	12
+273	val_273	2008-04-08	12
+273	val_273	2008-04-08	12
+273	val_273	2008-04-08	12
+274	val_274	2008-04-08	12
+275	val_275	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+278	val_278	2008-04-08	12
+278	val_278	2008-04-08	12
+280	val_280	2008-04-08	12
+280	val_280	2008-04-08	12
+281	val_281	2008-04-08	12
+281	val_281	2008-04-08	12
+282	val_282	2008-04-08	12
+282	val_282	2008-04-08	12
+283	val_283	2008-04-08	12
+284	val_284	2008-04-08	12
+285	val_285	2008-04-08	12
+286	val_286	2008-04-08	12
+287	val_287	2008-04-08	12
+288	val_288	2008-04-08	12
+288	val_288	2008-04-08	12
+289	val_289	2008-04-08	12
+291	val_291	2008-04-08	12
+292	val_292	2008-04-08	12
+296	val_296	2008-04-08	12
+298	val_298	2008-04-08	12
+298	val_298	2008-04-08	12
+298	val_298	2008-04-08	12
+302	val_302	2008-04-08	12
+305	val_305	2008-04-08	12
+306	val_306	2008-04-08	12
+307	val_307	2008-04-08	12
+307	val_307	2008-04-08	12
+308	val_308	2008-04-08	12
+309	val_309	2008-04-08	12
+309	val_309	2008-04-08	12
+310	val_310	2008-04-08	12
+311	val_311	2008-04-08	12
+311	val_311	2008-04-08	12
+311	val_311	2008-04-08	12
+315	val_315	2008-04-08	12
+316	val_316	2008-04-08	12
+316	val_316	2008-04-08	12
+316	val_316	2008-04-08	12
+317	val_317	2008-04-08	12
+317	val_317	2008-04-08	12
+318	val_318	2008-04-08	12
+318	val_318	2008-04-08	12
+318	val_318	2008-04-08	12
+321	val_321	2008-04-08	12
+321	val_321	2008-04-08	12
+322	val_322	2008-04-08	12
+322	val_322	2008-04-08	12
+323	val_323	2008-04-08	12
+325	val_325	2008-04-08	12
+325	val_325	2008-04-08	12
+327	val_327	2008-04-08	12
+327	val_327	2008-04-08	12
+327	val_327	2008-04-08	12
+331	val_331	2008-04-08	12
+331	val_331	2008-04-08	12
+332	val_332	2008-04-08	12
+333	val_333	2008-04-08	12
+333	val_333	2008-04-08	12
+335	val_335	2008-04-08	12
+336	val_336	2008-04-08	12
+338	val_338	2008-04-08	12
+339	val_339	2008-04-08	12
+341	val_341	2008-04-08	12
+342	val_342	2008-04-08	12
+342	val_342	2008-04-08	12
+344	val_344	2008-04-08	12
+344	val_344	2008-04-08	12
+345	val_345	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+351	val_351	2008-04-08	12
+353	val_353	2008-04-08	12
+353	val_353	2008-04-08	12
+356	val_356	2008-04-08	12
+360	val_360	2008-04-08	12
+362	val_362	2008-04-08	12
+364	val_364	2008-04-08	12
+365	val_365	2008-04-08	12
+366	val_366	2008-04-08	12
+367	val_367	2008-04-08	12
+367	val_367	2008-04-08	12
+368	val_368	2008-04-08	12
+369	val_369	2008-04-08	12
+369	val_369	2008-04-08	12
+369	val_369	2008-04-08	12
+373	val_373	2008-04-08	12
+374	val_374	2008-04-08	12
+375	val_375	2008-04-08	12
+377	val_377	2008-04-08	12
+378	val_378	2008-04-08	12
+379	val_379	2008-04-08	12
+382	val_382	2008-04-08	12
+382	val_382	2008-04-08	12
+384	val_384	2008-04-08	12
+384	val_384	2008-04-08	12
+384	val_384	2008-04-08	12
+386	val_386	2008-04-08	12
+389	val_389	2008-04-08	12
+392	val_392	2008-04-08	12
+393	val_393	2008-04-08	12
+394	val_394	2008-04-08	12
+395	val_395	2008-04-08	12
+395	val_395	2008-04-08	12
+396	val_396	2008-04-08	12
+396	val_396	2008-04-08	12
+396	val_396	2008-04-08	12
+397	val_397	2008-04-08	12
+397	val_397	2008-04-08	12
+399	val_399	2008-04-08	12
+399	val_399	2008-04-08	12
+400	val_400	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+402	val_402	2008-04-08	12
+403	val_403	2008-04-08	12
+403	val_403	2008-04-08	12
+403	val_403	2008-04-08	12
+404	val_404	2008-04-08	12
+404	val_404	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+407	val_407	2008-04-08	12
+409	val_409	2008-04-08	12
+409	val_409	2008-04-08	12
+409	val_409	2008-04-08	12
+411	val_411	2008-04-08	12
+413	val_413	2008-04-08	12
+413	val_413	2008-04-08	12
+414	val_414	2008-04-08	12
+414	val_414	2008-04-08	12
+417	val_417	2008-04-08	12
+417	val_417	2008-04-08	12
+417	val_417	2008-04-08	12
+418	val_418	2008-04-08	12
+419	val_419	2008-04-08	12
+421	val_421	2008-04-08	12
+424	val_424	2008-04-08	12
+424	val_424	2008-04-08	12
+427	val_427	2008-04-08	12
+429	val_429	2008-04-08	12
+429	val_429	2008-04-08	12
+430	val_430	2008-04-08	12
+430	val_430	2008-04-08	12
+430	val_430	2008-04-08	12
+431	val_431	2008-04-08	12
+431	val_431	2008-04-08	12
+431	val_431	2008-04-08	12
+432	val_432	2008-04-08	12
+435	val_435	2008-04-08	12
+436	val_436	2008-04-08	12
+437	val_437	2008-04-08	12
+438	val_438	2008-04-08	12
+438	val_438	2008-04-08	12
+438	val_438	2008-04-08	12
+439	val_439	2008-04-08	12
+439	val_439	2008-04-08	12
+443	val_443	2008-04-08	12
+444	val_444	2008-04-08	12
+446	val_446	2008-04-08	12
+448	val_448	2008-04-08	12
+449	val_449	2008-04-08	12
+452	val_452	2008-04-08	12
+453	val_453	2008-04-08	12
+454	val_454	2008-04-08	12
+454	val_454	2008-04-08	12
+454	val_454	2008-04-08	12
+455	val_455	2008-04-08	12
+457	val_457	2008-04-08	12
+458	val_458	2008-04-08	12
+458	val_458	2008-04-08	12
+459	val_459	2008-04-08	12
+459	val_459	2008-04-08	12
+460	val_460	2008-04-08	12
+462	val_462	2008-04-08	12
+462	val_462	2008-04-08	12
+463	val_463	2008-04-08	12
+463	val_463	2008-04-08	12
+466	val_466	2008-04-08	12
+466	val_466	2008-04-08	12
+466	val_466	2008-04-08	12
+467	val_467	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+470	val_470	2008-04-08	12
+472	val_472	2008-04-08	12
+475	val_475	2008-04-08	12
+477	val_477	2008-04-08	12
+478	val_478	2008-04-08	12
+478	val_478	2008-04-08	12
+479	val_479	2008-04-08	12
+480	val_480	2008-04-08	12
+480	val_480	2008-04-08	12
+480	val_480	2008-04-08	12
+481	val_481	2008-04-08	12
+482	val_482	2008-04-08	12
+483	val_483	2008-04-08	12
+484	val_484	2008-04-08	12
+485	val_485	2008-04-08	12
+487	val_487	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+490	val_490	2008-04-08	12
+491	val_491	2008-04-08	12
+492	val_492	2008-04-08	12
+492	val_492	2008-04-08	12
+493	val_493	2008-04-08	12
+494	val_494	2008-04-08	12
+495	val_495	2008-04-08	12
+496	val_496	2008-04-08	12
+497	val_497	2008-04-08	12
+498	val_498	2008-04-08	12
+498	val_498	2008-04-08	12
+498	val_498	2008-04-08	12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 b/sql/hive/src/test/resources/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66
deleted file mode 100644
index d7a8f25b41301ec8d6a1bd3d230e31a65586444e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66
+++ /dev/null
@@ -1,1000 +0,0 @@
-238	val_238	2008-04-08	11
-86	val_86	2008-04-08	11
-311	val_311	2008-04-08	11
-27	val_27	2008-04-08	11
-165	val_165	2008-04-08	11
-409	val_409	2008-04-08	11
-255	val_255	2008-04-08	11
-278	val_278	2008-04-08	11
-98	val_98	2008-04-08	11
-484	val_484	2008-04-08	11
-265	val_265	2008-04-08	11
-193	val_193	2008-04-08	11
-401	val_401	2008-04-08	11
-150	val_150	2008-04-08	11
-273	val_273	2008-04-08	11
-224	val_224	2008-04-08	11
-369	val_369	2008-04-08	11
-66	val_66	2008-04-08	11
-128	val_128	2008-04-08	11
-213	val_213	2008-04-08	11
-146	val_146	2008-04-08	11
-406	val_406	2008-04-08	11
-429	val_429	2008-04-08	11
-374	val_374	2008-04-08	11
-152	val_152	2008-04-08	11
-469	val_469	2008-04-08	11
-145	val_145	2008-04-08	11
-495	val_495	2008-04-08	11
-37	val_37	2008-04-08	11
-327	val_327	2008-04-08	11
-281	val_281	2008-04-08	11
-277	val_277	2008-04-08	11
-209	val_209	2008-04-08	11
-15	val_15	2008-04-08	11
-82	val_82	2008-04-08	11
-403	val_403	2008-04-08	11
-166	val_166	2008-04-08	11
-417	val_417	2008-04-08	11
-430	val_430	2008-04-08	11
-252	val_252	2008-04-08	11
-292	val_292	2008-04-08	11
-219	val_219	2008-04-08	11
-287	val_287	2008-04-08	11
-153	val_153	2008-04-08	11
-193	val_193	2008-04-08	11
-338	val_338	2008-04-08	11
-446	val_446	2008-04-08	11
-459	val_459	2008-04-08	11
-394	val_394	2008-04-08	11
-237	val_237	2008-04-08	11
-482	val_482	2008-04-08	11
-174	val_174	2008-04-08	11
-413	val_413	2008-04-08	11
-494	val_494	2008-04-08	11
-207	val_207	2008-04-08	11
-199	val_199	2008-04-08	11
-466	val_466	2008-04-08	11
-208	val_208	2008-04-08	11
-174	val_174	2008-04-08	11
-399	val_399	2008-04-08	11
-396	val_396	2008-04-08	11
-247	val_247	2008-04-08	11
-417	val_417	2008-04-08	11
-489	val_489	2008-04-08	11
-162	val_162	2008-04-08	11
-377	val_377	2008-04-08	11
-397	val_397	2008-04-08	11
-309	val_309	2008-04-08	11
-365	val_365	2008-04-08	11
-266	val_266	2008-04-08	11
-439	val_439	2008-04-08	11
-342	val_342	2008-04-08	11
-367	val_367	2008-04-08	11
-325	val_325	2008-04-08	11
-167	val_167	2008-04-08	11
-195	val_195	2008-04-08	11
-475	val_475	2008-04-08	11
-17	val_17	2008-04-08	11
-113	val_113	2008-04-08	11
-155	val_155	2008-04-08	11
-203	val_203	2008-04-08	11
-339	val_339	2008-04-08	11
-0	val_0	2008-04-08	11
-455	val_455	2008-04-08	11
-128	val_128	2008-04-08	11
-311	val_311	2008-04-08	11
-316	val_316	2008-04-08	11
-57	val_57	2008-04-08	11
-302	val_302	2008-04-08	11
-205	val_205	2008-04-08	11
-149	val_149	2008-04-08	11
-438	val_438	2008-04-08	11
-345	val_345	2008-04-08	11
-129	val_129	2008-04-08	11
-170	val_170	2008-04-08	11
-20	val_20	2008-04-08	11
-489	val_489	2008-04-08	11
-157	val_157	2008-04-08	11
-378	val_378	2008-04-08	11
-221	val_221	2008-04-08	11
-92	val_92	2008-04-08	11
-111	val_111	2008-04-08	11
-47	val_47	2008-04-08	11
-72	val_72	2008-04-08	11
-4	val_4	2008-04-08	11
-280	val_280	2008-04-08	11
-35	val_35	2008-04-08	11
-427	val_427	2008-04-08	11
-277	val_277	2008-04-08	11
-208	val_208	2008-04-08	11
-356	val_356	2008-04-08	11
-399	val_399	2008-04-08	11
-169	val_169	2008-04-08	11
-382	val_382	2008-04-08	11
-498	val_498	2008-04-08	11
-125	val_125	2008-04-08	11
-386	val_386	2008-04-08	11
-437	val_437	2008-04-08	11
-469	val_469	2008-04-08	11
-192	val_192	2008-04-08	11
-286	val_286	2008-04-08	11
-187	val_187	2008-04-08	11
-176	val_176	2008-04-08	11
-54	val_54	2008-04-08	11
-459	val_459	2008-04-08	11
-51	val_51	2008-04-08	11
-138	val_138	2008-04-08	11
-103	val_103	2008-04-08	11
-239	val_239	2008-04-08	11
-213	val_213	2008-04-08	11
-216	val_216	2008-04-08	11
-430	val_430	2008-04-08	11
-278	val_278	2008-04-08	11
-176	val_176	2008-04-08	11
-289	val_289	2008-04-08	11
-221	val_221	2008-04-08	11
-65	val_65	2008-04-08	11
-318	val_318	2008-04-08	11
-332	val_332	2008-04-08	11
-311	val_311	2008-04-08	11
-275	val_275	2008-04-08	11
-137	val_137	2008-04-08	11
-241	val_241	2008-04-08	11
-83	val_83	2008-04-08	11
-333	val_333	2008-04-08	11
-180	val_180	2008-04-08	11
-284	val_284	2008-04-08	11
-12	val_12	2008-04-08	11
-230	val_230	2008-04-08	11
-181	val_181	2008-04-08	11
-67	val_67	2008-04-08	11
-260	val_260	2008-04-08	11
-404	val_404	2008-04-08	11
-384	val_384	2008-04-08	11
-489	val_489	2008-04-08	11
-353	val_353	2008-04-08	11
-373	val_373	2008-04-08	11
-272	val_272	2008-04-08	11
-138	val_138	2008-04-08	11
-217	val_217	2008-04-08	11
-84	val_84	2008-04-08	11
-348	val_348	2008-04-08	11
-466	val_466	2008-04-08	11
-58	val_58	2008-04-08	11
-8	val_8	2008-04-08	11
-411	val_411	2008-04-08	11
-230	val_230	2008-04-08	11
-208	val_208	2008-04-08	11
-348	val_348	2008-04-08	11
-24	val_24	2008-04-08	11
-463	val_463	2008-04-08	11
-431	val_431	2008-04-08	11
-179	val_179	2008-04-08	11
-172	val_172	2008-04-08	11
-42	val_42	2008-04-08	11
-129	val_129	2008-04-08	11
-158	val_158	2008-04-08	11
-119	val_119	2008-04-08	11
-496	val_496	2008-04-08	11
-0	val_0	2008-04-08	11
-322	val_322	2008-04-08	11
-197	val_197	2008-04-08	11
-468	val_468	2008-04-08	11
-393	val_393	2008-04-08	11
-454	val_454	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-199	val_199	2008-04-08	11
-191	val_191	2008-04-08	11
-418	val_418	2008-04-08	11
-96	val_96	2008-04-08	11
-26	val_26	2008-04-08	11
-165	val_165	2008-04-08	11
-327	val_327	2008-04-08	11
-230	val_230	2008-04-08	11
-205	val_205	2008-04-08	11
-120	val_120	2008-04-08	11
-131	val_131	2008-04-08	11
-51	val_51	2008-04-08	11
-404	val_404	2008-04-08	11
-43	val_43	2008-04-08	11
-436	val_436	2008-04-08	11
-156	val_156	2008-04-08	11
-469	val_469	2008-04-08	11
-468	val_468	2008-04-08	11
-308	val_308	2008-04-08	11
-95	val_95	2008-04-08	11
-196	val_196	2008-04-08	11
-288	val_288	2008-04-08	11
-481	val_481	2008-04-08	11
-457	val_457	2008-04-08	11
-98	val_98	2008-04-08	11
-282	val_282	2008-04-08	11
-197	val_197	2008-04-08	11
-187	val_187	2008-04-08	11
-318	val_318	2008-04-08	11
-318	val_318	2008-04-08	11
-409	val_409	2008-04-08	11
-470	val_470	2008-04-08	11
-137	val_137	2008-04-08	11
-369	val_369	2008-04-08	11
-316	val_316	2008-04-08	11
-169	val_169	2008-04-08	11
-413	val_413	2008-04-08	11
-85	val_85	2008-04-08	11
-77	val_77	2008-04-08	11
-0	val_0	2008-04-08	11
-490	val_490	2008-04-08	11
-87	val_87	2008-04-08	11
-364	val_364	2008-04-08	11
-179	val_179	2008-04-08	11
-118	val_118	2008-04-08	11
-134	val_134	2008-04-08	11
-395	val_395	2008-04-08	11
-282	val_282	2008-04-08	11
-138	val_138	2008-04-08	11
-238	val_238	2008-04-08	11
-419	val_419	2008-04-08	11
-15	val_15	2008-04-08	11
-118	val_118	2008-04-08	11
-72	val_72	2008-04-08	11
-90	val_90	2008-04-08	11
-307	val_307	2008-04-08	11
-19	val_19	2008-04-08	11
-435	val_435	2008-04-08	11
-10	val_10	2008-04-08	11
-277	val_277	2008-04-08	11
-273	val_273	2008-04-08	11
-306	val_306	2008-04-08	11
-224	val_224	2008-04-08	11
-309	val_309	2008-04-08	11
-389	val_389	2008-04-08	11
-327	val_327	2008-04-08	11
-242	val_242	2008-04-08	11
-369	val_369	2008-04-08	11
-392	val_392	2008-04-08	11
-272	val_272	2008-04-08	11
-331	val_331	2008-04-08	11
-401	val_401	2008-04-08	11
-242	val_242	2008-04-08	11
-452	val_452	2008-04-08	11
-177	val_177	2008-04-08	11
-226	val_226	2008-04-08	11
-5	val_5	2008-04-08	11
-497	val_497	2008-04-08	11
-402	val_402	2008-04-08	11
-396	val_396	2008-04-08	11
-317	val_317	2008-04-08	11
-395	val_395	2008-04-08	11
-58	val_58	2008-04-08	11
-35	val_35	2008-04-08	11
-336	val_336	2008-04-08	11
-95	val_95	2008-04-08	11
-11	val_11	2008-04-08	11
-168	val_168	2008-04-08	11
-34	val_34	2008-04-08	11
-229	val_229	2008-04-08	11
-233	val_233	2008-04-08	11
-143	val_143	2008-04-08	11
-472	val_472	2008-04-08	11
-322	val_322	2008-04-08	11
-498	val_498	2008-04-08	11
-160	val_160	2008-04-08	11
-195	val_195	2008-04-08	11
-42	val_42	2008-04-08	11
-321	val_321	2008-04-08	11
-430	val_430	2008-04-08	11
-119	val_119	2008-04-08	11
-489	val_489	2008-04-08	11
-458	val_458	2008-04-08	11
-78	val_78	2008-04-08	11
-76	val_76	2008-04-08	11
-41	val_41	2008-04-08	11
-223	val_223	2008-04-08	11
-492	val_492	2008-04-08	11
-149	val_149	2008-04-08	11
-449	val_449	2008-04-08	11
-218	val_218	2008-04-08	11
-228	val_228	2008-04-08	11
-138	val_138	2008-04-08	11
-453	val_453	2008-04-08	11
-30	val_30	2008-04-08	11
-209	val_209	2008-04-08	11
-64	val_64	2008-04-08	11
-468	val_468	2008-04-08	11
-76	val_76	2008-04-08	11
-74	val_74	2008-04-08	11
-342	val_342	2008-04-08	11
-69	val_69	2008-04-08	11
-230	val_230	2008-04-08	11
-33	val_33	2008-04-08	11
-368	val_368	2008-04-08	11
-103	val_103	2008-04-08	11
-296	val_296	2008-04-08	11
-113	val_113	2008-04-08	11
-216	val_216	2008-04-08	11
-367	val_367	2008-04-08	11
-344	val_344	2008-04-08	11
-167	val_167	2008-04-08	11
-274	val_274	2008-04-08	11
-219	val_219	2008-04-08	11
-239	val_239	2008-04-08	11
-485	val_485	2008-04-08	11
-116	val_116	2008-04-08	11
-223	val_223	2008-04-08	11
-256	val_256	2008-04-08	11
-263	val_263	2008-04-08	11
-70	val_70	2008-04-08	11
-487	val_487	2008-04-08	11
-480	val_480	2008-04-08	11
-401	val_401	2008-04-08	11
-288	val_288	2008-04-08	11
-191	val_191	2008-04-08	11
-5	val_5	2008-04-08	11
-244	val_244	2008-04-08	11
-438	val_438	2008-04-08	11
-128	val_128	2008-04-08	11
-467	val_467	2008-04-08	11
-432	val_432	2008-04-08	11
-202	val_202	2008-04-08	11
-316	val_316	2008-04-08	11
-229	val_229	2008-04-08	11
-469	val_469	2008-04-08	11
-463	val_463	2008-04-08	11
-280	val_280	2008-04-08	11
-2	val_2	2008-04-08	11
-35	val_35	2008-04-08	11
-283	val_283	2008-04-08	11
-331	val_331	2008-04-08	11
-235	val_235	2008-04-08	11
-80	val_80	2008-04-08	11
-44	val_44	2008-04-08	11
-193	val_193	2008-04-08	11
-321	val_321	2008-04-08	11
-335	val_335	2008-04-08	11
-104	val_104	2008-04-08	11
-466	val_466	2008-04-08	11
-366	val_366	2008-04-08	11
-175	val_175	2008-04-08	11
-403	val_403	2008-04-08	11
-483	val_483	2008-04-08	11
-53	val_53	2008-04-08	11
-105	val_105	2008-04-08	11
-257	val_257	2008-04-08	11
-406	val_406	2008-04-08	11
-409	val_409	2008-04-08	11
-190	val_190	2008-04-08	11
-406	val_406	2008-04-08	11
-401	val_401	2008-04-08	11
-114	val_114	2008-04-08	11
-258	val_258	2008-04-08	11
-90	val_90	2008-04-08	11
-203	val_203	2008-04-08	11
-262	val_262	2008-04-08	11
-348	val_348	2008-04-08	11
-424	val_424	2008-04-08	11
-12	val_12	2008-04-08	11
-396	val_396	2008-04-08	11
-201	val_201	2008-04-08	11
-217	val_217	2008-04-08	11
-164	val_164	2008-04-08	11
-431	val_431	2008-04-08	11
-454	val_454	2008-04-08	11
-478	val_478	2008-04-08	11
-298	val_298	2008-04-08	11
-125	val_125	2008-04-08	11
-431	val_431	2008-04-08	11
-164	val_164	2008-04-08	11
-424	val_424	2008-04-08	11
-187	val_187	2008-04-08	11
-382	val_382	2008-04-08	11
-5	val_5	2008-04-08	11
-70	val_70	2008-04-08	11
-397	val_397	2008-04-08	11
-480	val_480	2008-04-08	11
-291	val_291	2008-04-08	11
-24	val_24	2008-04-08	11
-351	val_351	2008-04-08	11
-255	val_255	2008-04-08	11
-104	val_104	2008-04-08	11
-70	val_70	2008-04-08	11
-163	val_163	2008-04-08	11
-438	val_438	2008-04-08	11
-119	val_119	2008-04-08	11
-414	val_414	2008-04-08	11
-200	val_200	2008-04-08	11
-491	val_491	2008-04-08	11
-237	val_237	2008-04-08	11
-439	val_439	2008-04-08	11
-360	val_360	2008-04-08	11
-248	val_248	2008-04-08	11
-479	val_479	2008-04-08	11
-305	val_305	2008-04-08	11
-417	val_417	2008-04-08	11
-199	val_199	2008-04-08	11
-444	val_444	2008-04-08	11
-120	val_120	2008-04-08	11
-429	val_429	2008-04-08	11
-169	val_169	2008-04-08	11
-443	val_443	2008-04-08	11
-323	val_323	2008-04-08	11
-325	val_325	2008-04-08	11
-277	val_277	2008-04-08	11
-230	val_230	2008-04-08	11
-478	val_478	2008-04-08	11
-178	val_178	2008-04-08	11
-468	val_468	2008-04-08	11
-310	val_310	2008-04-08	11
-317	val_317	2008-04-08	11
-333	val_333	2008-04-08	11
-493	val_493	2008-04-08	11
-460	val_460	2008-04-08	11
-207	val_207	2008-04-08	11
-249	val_249	2008-04-08	11
-265	val_265	2008-04-08	11
-480	val_480	2008-04-08	11
-83	val_83	2008-04-08	11
-136	val_136	2008-04-08	11
-353	val_353	2008-04-08	11
-172	val_172	2008-04-08	11
-214	val_214	2008-04-08	11
-462	val_462	2008-04-08	11
-233	val_233	2008-04-08	11
-406	val_406	2008-04-08	11
-133	val_133	2008-04-08	11
-175	val_175	2008-04-08	11
-189	val_189	2008-04-08	11
-454	val_454	2008-04-08	11
-375	val_375	2008-04-08	11
-401	val_401	2008-04-08	11
-421	val_421	2008-04-08	11
-407	val_407	2008-04-08	11
-384	val_384	2008-04-08	11
-256	val_256	2008-04-08	11
-26	val_26	2008-04-08	11
-134	val_134	2008-04-08	11
-67	val_67	2008-04-08	11
-384	val_384	2008-04-08	11
-379	val_379	2008-04-08	11
-18	val_18	2008-04-08	11
-462	val_462	2008-04-08	11
-492	val_492	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-9	val_9	2008-04-08	11
-341	val_341	2008-04-08	11
-498	val_498	2008-04-08	11
-146	val_146	2008-04-08	11
-458	val_458	2008-04-08	11
-362	val_362	2008-04-08	11
-186	val_186	2008-04-08	11
-285	val_285	2008-04-08	11
-348	val_348	2008-04-08	11
-167	val_167	2008-04-08	11
-18	val_18	2008-04-08	11
-273	val_273	2008-04-08	11
-183	val_183	2008-04-08	11
-281	val_281	2008-04-08	11
-344	val_344	2008-04-08	11
-97	val_97	2008-04-08	11
-469	val_469	2008-04-08	11
-315	val_315	2008-04-08	11
-84	val_84	2008-04-08	11
-28	val_28	2008-04-08	11
-37	val_37	2008-04-08	11
-448	val_448	2008-04-08	11
-152	val_152	2008-04-08	11
-348	val_348	2008-04-08	11
-307	val_307	2008-04-08	11
-194	val_194	2008-04-08	11
-414	val_414	2008-04-08	11
-477	val_477	2008-04-08	11
-222	val_222	2008-04-08	11
-126	val_126	2008-04-08	11
-90	val_90	2008-04-08	11
-169	val_169	2008-04-08	11
-403	val_403	2008-04-08	11
-400	val_400	2008-04-08	11
-200	val_200	2008-04-08	11
-97	val_97	2008-04-08	11
-238	val_238	2008-04-08	12
-86	val_86	2008-04-08	12
-311	val_311	2008-04-08	12
-27	val_27	2008-04-08	12
-165	val_165	2008-04-08	12
-409	val_409	2008-04-08	12
-255	val_255	2008-04-08	12
-278	val_278	2008-04-08	12
-98	val_98	2008-04-08	12
-484	val_484	2008-04-08	12
-265	val_265	2008-04-08	12
-193	val_193	2008-04-08	12
-401	val_401	2008-04-08	12
-150	val_150	2008-04-08	12
-273	val_273	2008-04-08	12
-224	val_224	2008-04-08	12
-369	val_369	2008-04-08	12
-66	val_66	2008-04-08	12
-128	val_128	2008-04-08	12
-213	val_213	2008-04-08	12
-146	val_146	2008-04-08	12
-406	val_406	2008-04-08	12
-429	val_429	2008-04-08	12
-374	val_374	2008-04-08	12
-152	val_152	2008-04-08	12
-469	val_469	2008-04-08	12
-145	val_145	2008-04-08	12
-495	val_495	2008-04-08	12
-37	val_37	2008-04-08	12
-327	val_327	2008-04-08	12
-281	val_281	2008-04-08	12
-277	val_277	2008-04-08	12
-209	val_209	2008-04-08	12
-15	val_15	2008-04-08	12
-82	val_82	2008-04-08	12
-403	val_403	2008-04-08	12
-166	val_166	2008-04-08	12
-417	val_417	2008-04-08	12
-430	val_430	2008-04-08	12
-252	val_252	2008-04-08	12
-292	val_292	2008-04-08	12
-219	val_219	2008-04-08	12
-287	val_287	2008-04-08	12
-153	val_153	2008-04-08	12
-193	val_193	2008-04-08	12
-338	val_338	2008-04-08	12
-446	val_446	2008-04-08	12
-459	val_459	2008-04-08	12
-394	val_394	2008-04-08	12
-237	val_237	2008-04-08	12
-482	val_482	2008-04-08	12
-174	val_174	2008-04-08	12
-413	val_413	2008-04-08	12
-494	val_494	2008-04-08	12
-207	val_207	2008-04-08	12
-199	val_199	2008-04-08	12
-466	val_466	2008-04-08	12
-208	val_208	2008-04-08	12
-174	val_174	2008-04-08	12
-399	val_399	2008-04-08	12
-396	val_396	2008-04-08	12
-247	val_247	2008-04-08	12
-417	val_417	2008-04-08	12
-489	val_489	2008-04-08	12
-162	val_162	2008-04-08	12
-377	val_377	2008-04-08	12
-397	val_397	2008-04-08	12
-309	val_309	2008-04-08	12
-365	val_365	2008-04-08	12
-266	val_266	2008-04-08	12
-439	val_439	2008-04-08	12
-342	val_342	2008-04-08	12
-367	val_367	2008-04-08	12
-325	val_325	2008-04-08	12
-167	val_167	2008-04-08	12
-195	val_195	2008-04-08	12
-475	val_475	2008-04-08	12
-17	val_17	2008-04-08	12
-113	val_113	2008-04-08	12
-155	val_155	2008-04-08	12
-203	val_203	2008-04-08	12
-339	val_339	2008-04-08	12
-0	val_0	2008-04-08	12
-455	val_455	2008-04-08	12
-128	val_128	2008-04-08	12
-311	val_311	2008-04-08	12
-316	val_316	2008-04-08	12
-57	val_57	2008-04-08	12
-302	val_302	2008-04-08	12
-205	val_205	2008-04-08	12
-149	val_149	2008-04-08	12
-438	val_438	2008-04-08	12
-345	val_345	2008-04-08	12
-129	val_129	2008-04-08	12
-170	val_170	2008-04-08	12
-20	val_20	2008-04-08	12
-489	val_489	2008-04-08	12
-157	val_157	2008-04-08	12
-378	val_378	2008-04-08	12
-221	val_221	2008-04-08	12
-92	val_92	2008-04-08	12
-111	val_111	2008-04-08	12
-47	val_47	2008-04-08	12
-72	val_72	2008-04-08	12
-4	val_4	2008-04-08	12
-280	val_280	2008-04-08	12
-35	val_35	2008-04-08	12
-427	val_427	2008-04-08	12
-277	val_277	2008-04-08	12
-208	val_208	2008-04-08	12
-356	val_356	2008-04-08	12
-399	val_399	2008-04-08	12
-169	val_169	2008-04-08	12
-382	val_382	2008-04-08	12
-498	val_498	2008-04-08	12
-125	val_125	2008-04-08	12
-386	val_386	2008-04-08	12
-437	val_437	2008-04-08	12
-469	val_469	2008-04-08	12
-192	val_192	2008-04-08	12
-286	val_286	2008-04-08	12
-187	val_187	2008-04-08	12
-176	val_176	2008-04-08	12
-54	val_54	2008-04-08	12
-459	val_459	2008-04-08	12
-51	val_51	2008-04-08	12
-138	val_138	2008-04-08	12
-103	val_103	2008-04-08	12
-239	val_239	2008-04-08	12
-213	val_213	2008-04-08	12
-216	val_216	2008-04-08	12
-430	val_430	2008-04-08	12
-278	val_278	2008-04-08	12
-176	val_176	2008-04-08	12
-289	val_289	2008-04-08	12
-221	val_221	2008-04-08	12
-65	val_65	2008-04-08	12
-318	val_318	2008-04-08	12
-332	val_332	2008-04-08	12
-311	val_311	2008-04-08	12
-275	val_275	2008-04-08	12
-137	val_137	2008-04-08	12
-241	val_241	2008-04-08	12
-83	val_83	2008-04-08	12
-333	val_333	2008-04-08	12
-180	val_180	2008-04-08	12
-284	val_284	2008-04-08	12
-12	val_12	2008-04-08	12
-230	val_230	2008-04-08	12
-181	val_181	2008-04-08	12
-67	val_67	2008-04-08	12
-260	val_260	2008-04-08	12
-404	val_404	2008-04-08	12
-384	val_384	2008-04-08	12
-489	val_489	2008-04-08	12
-353	val_353	2008-04-08	12
-373	val_373	2008-04-08	12
-272	val_272	2008-04-08	12
-138	val_138	2008-04-08	12
-217	val_217	2008-04-08	12
-84	val_84	2008-04-08	12
-348	val_348	2008-04-08	12
-466	val_466	2008-04-08	12
-58	val_58	2008-04-08	12
-8	val_8	2008-04-08	12
-411	val_411	2008-04-08	12
-230	val_230	2008-04-08	12
-208	val_208	2008-04-08	12
-348	val_348	2008-04-08	12
-24	val_24	2008-04-08	12
-463	val_463	2008-04-08	12
-431	val_431	2008-04-08	12
-179	val_179	2008-04-08	12
-172	val_172	2008-04-08	12
-42	val_42	2008-04-08	12
-129	val_129	2008-04-08	12
-158	val_158	2008-04-08	12
-119	val_119	2008-04-08	12
-496	val_496	2008-04-08	12
-0	val_0	2008-04-08	12
-322	val_322	2008-04-08	12
-197	val_197	2008-04-08	12
-468	val_468	2008-04-08	12
-393	val_393	2008-04-08	12
-454	val_454	2008-04-08	12
-100	val_100	2008-04-08	12
-298	val_298	2008-04-08	12
-199	val_199	2008-04-08	12
-191	val_191	2008-04-08	12
-418	val_418	2008-04-08	12
-96	val_96	2008-04-08	12
-26	val_26	2008-04-08	12
-165	val_165	2008-04-08	12
-327	val_327	2008-04-08	12
-230	val_230	2008-04-08	12
-205	val_205	2008-04-08	12
-120	val_120	2008-04-08	12
-131	val_131	2008-04-08	12
-51	val_51	2008-04-08	12
-404	val_404	2008-04-08	12
-43	val_43	2008-04-08	12
-436	val_436	2008-04-08	12
-156	val_156	2008-04-08	12
-469	val_469	2008-04-08	12
-468	val_468	2008-04-08	12
-308	val_308	2008-04-08	12
-95	val_95	2008-04-08	12
-196	val_196	2008-04-08	12
-288	val_288	2008-04-08	12
-481	val_481	2008-04-08	12
-457	val_457	2008-04-08	12
-98	val_98	2008-04-08	12
-282	val_282	2008-04-08	12
-197	val_197	2008-04-08	12
-187	val_187	2008-04-08	12
-318	val_318	2008-04-08	12
-318	val_318	2008-04-08	12
-409	val_409	2008-04-08	12
-470	val_470	2008-04-08	12
-137	val_137	2008-04-08	12
-369	val_369	2008-04-08	12
-316	val_316	2008-04-08	12
-169	val_169	2008-04-08	12
-413	val_413	2008-04-08	12
-85	val_85	2008-04-08	12
-77	val_77	2008-04-08	12
-0	val_0	2008-04-08	12
-490	val_490	2008-04-08	12
-87	val_87	2008-04-08	12
-364	val_364	2008-04-08	12
-179	val_179	2008-04-08	12
-118	val_118	2008-04-08	12
-134	val_134	2008-04-08	12
-395	val_395	2008-04-08	12
-282	val_282	2008-04-08	12
-138	val_138	2008-04-08	12
-238	val_238	2008-04-08	12
-419	val_419	2008-04-08	12
-15	val_15	2008-04-08	12
-118	val_118	2008-04-08	12
-72	val_72	2008-04-08	12
-90	val_90	2008-04-08	12
-307	val_307	2008-04-08	12
-19	val_19	2008-04-08	12
-435	val_435	2008-04-08	12
-10	val_10	2008-04-08	12
-277	val_277	2008-04-08	12
-273	val_273	2008-04-08	12
-306	val_306	2008-04-08	12
-224	val_224	2008-04-08	12
-309	val_309	2008-04-08	12
-389	val_389	2008-04-08	12
-327	val_327	2008-04-08	12
-242	val_242	2008-04-08	12
-369	val_369	2008-04-08	12
-392	val_392	2008-04-08	12
-272	val_272	2008-04-08	12
-331	val_331	2008-04-08	12
-401	val_401	2008-04-08	12
-242	val_242	2008-04-08	12
-452	val_452	2008-04-08	12
-177	val_177	2008-04-08	12
-226	val_226	2008-04-08	12
-5	val_5	2008-04-08	12
-497	val_497	2008-04-08	12
-402	val_402	2008-04-08	12
-396	val_396	2008-04-08	12
-317	val_317	2008-04-08	12
-395	val_395	2008-04-08	12
-58	val_58	2008-04-08	12
-35	val_35	2008-04-08	12
-336	val_336	2008-04-08	12
-95	val_95	2008-04-08	12
-11	val_11	2008-04-08	12
-168	val_168	2008-04-08	12
-34	val_34	2008-04-08	12
-229	val_229	2008-04-08	12
-233	val_233	2008-04-08	12
-143	val_143	2008-04-08	12
-472	val_472	2008-04-08	12
-322	val_322	2008-04-08	12
-498	val_498	2008-04-08	12
-160	val_160	2008-04-08	12
-195	val_195	2008-04-08	12
-42	val_42	2008-04-08	12
-321	val_321	2008-04-08	12
-430	val_430	2008-04-08	12
-119	val_119	2008-04-08	12
-489	val_489	2008-04-08	12
-458	val_458	2008-04-08	12
-78	val_78	2008-04-08	12
-76	val_76	2008-04-08	12
-41	val_41	2008-04-08	12
-223	val_223	2008-04-08	12
-492	val_492	2008-04-08	12
-149	val_149	2008-04-08	12
-449	val_449	2008-04-08	12
-218	val_218	2008-04-08	12
-228	val_228	2008-04-08	12
-138	val_138	2008-04-08	12
-453	val_453	2008-04-08	12
-30	val_30	2008-04-08	12
-209	val_209	2008-04-08	12
-64	val_64	2008-04-08	12
-468	val_468	2008-04-08	12
-76	val_76	2008-04-08	12
-74	val_74	2008-04-08	12
-342	val_342	2008-04-08	12
-69	val_69	2008-04-08	12
-230	val_230	2008-04-08	12
-33	val_33	2008-04-08	12
-368	val_368	2008-04-08	12
-103	val_103	2008-04-08	12
-296	val_296	2008-04-08	12
-113	val_113	2008-04-08	12
-216	val_216	2008-04-08	12
-367	val_367	2008-04-08	12
-344	val_344	2008-04-08	12
-167	val_167	2008-04-08	12
-274	val_274	2008-04-08	12
-219	val_219	2008-04-08	12
-239	val_239	2008-04-08	12
-485	val_485	2008-04-08	12
-116	val_116	2008-04-08	12
-223	val_223	2008-04-08	12
-256	val_256	2008-04-08	12
-263	val_263	2008-04-08	12
-70	val_70	2008-04-08	12
-487	val_487	2008-04-08	12
-480	val_480	2008-04-08	12
-401	val_401	2008-04-08	12
-288	val_288	2008-04-08	12
-191	val_191	2008-04-08	12
-5	val_5	2008-04-08	12
-244	val_244	2008-04-08	12
-438	val_438	2008-04-08	12
-128	val_128	2008-04-08	12
-467	val_467	2008-04-08	12
-432	val_432	2008-04-08	12
-202	val_202	2008-04-08	12
-316	val_316	2008-04-08	12
-229	val_229	2008-04-08	12
-469	val_469	2008-04-08	12
-463	val_463	2008-04-08	12
-280	val_280	2008-04-08	12
-2	val_2	2008-04-08	12
-35	val_35	2008-04-08	12
-283	val_283	2008-04-08	12
-331	val_331	2008-04-08	12
-235	val_235	2008-04-08	12
-80	val_80	2008-04-08	12
-44	val_44	2008-04-08	12
-193	val_193	2008-04-08	12
-321	val_321	2008-04-08	12
-335	val_335	2008-04-08	12
-104	val_104	2008-04-08	12
-466	val_466	2008-04-08	12
-366	val_366	2008-04-08	12
-175	val_175	2008-04-08	12
-403	val_403	2008-04-08	12
-483	val_483	2008-04-08	12
-53	val_53	2008-04-08	12
-105	val_105	2008-04-08	12
-257	val_257	2008-04-08	12
-406	val_406	2008-04-08	12
-409	val_409	2008-04-08	12
-190	val_190	2008-04-08	12
-406	val_406	2008-04-08	12
-401	val_401	2008-04-08	12
-114	val_114	2008-04-08	12
-258	val_258	2008-04-08	12
-90	val_90	2008-04-08	12
-203	val_203	2008-04-08	12
-262	val_262	2008-04-08	12
-348	val_348	2008-04-08	12
-424	val_424	2008-04-08	12
-12	val_12	2008-04-08	12
-396	val_396	2008-04-08	12
-201	val_201	2008-04-08	12
-217	val_217	2008-04-08	12
-164	val_164	2008-04-08	12
-431	val_431	2008-04-08	12
-454	val_454	2008-04-08	12
-478	val_478	2008-04-08	12
-298	val_298	2008-04-08	12
-125	val_125	2008-04-08	12
-431	val_431	2008-04-08	12
-164	val_164	2008-04-08	12
-424	val_424	2008-04-08	12
-187	val_187	2008-04-08	12
-382	val_382	2008-04-08	12
-5	val_5	2008-04-08	12
-70	val_70	2008-04-08	12
-397	val_397	2008-04-08	12
-480	val_480	2008-04-08	12
-291	val_291	2008-04-08	12
-24	val_24	2008-04-08	12
-351	val_351	2008-04-08	12
-255	val_255	2008-04-08	12
-104	val_104	2008-04-08	12
-70	val_70	2008-04-08	12
-163	val_163	2008-04-08	12
-438	val_438	2008-04-08	12
-119	val_119	2008-04-08	12
-414	val_414	2008-04-08	12
-200	val_200	2008-04-08	12
-491	val_491	2008-04-08	12
-237	val_237	2008-04-08	12
-439	val_439	2008-04-08	12
-360	val_360	2008-04-08	12
-248	val_248	2008-04-08	12
-479	val_479	2008-04-08	12
-305	val_305	2008-04-08	12
-417	val_417	2008-04-08	12
-199	val_199	2008-04-08	12
-444	val_444	2008-04-08	12
-120	val_120	2008-04-08	12
-429	val_429	2008-04-08	12
-169	val_169	2008-04-08	12
-443	val_443	2008-04-08	12
-323	val_323	2008-04-08	12
-325	val_325	2008-04-08	12
-277	val_277	2008-04-08	12
-230	val_230	2008-04-08	12
-478	val_478	2008-04-08	12
-178	val_178	2008-04-08	12
-468	val_468	2008-04-08	12
-310	val_310	2008-04-08	12
-317	val_317	2008-04-08	12
-333	val_333	2008-04-08	12
-493	val_493	2008-04-08	12
-460	val_460	2008-04-08	12
-207	val_207	2008-04-08	12
-249	val_249	2008-04-08	12
-265	val_265	2008-04-08	12
-480	val_480	2008-04-08	12
-83	val_83	2008-04-08	12
-136	val_136	2008-04-08	12
-353	val_353	2008-04-08	12
-172	val_172	2008-04-08	12
-214	val_214	2008-04-08	12
-462	val_462	2008-04-08	12
-233	val_233	2008-04-08	12
-406	val_406	2008-04-08	12
-133	val_133	2008-04-08	12
-175	val_175	2008-04-08	12
-189	val_189	2008-04-08	12
-454	val_454	2008-04-08	12
-375	val_375	2008-04-08	12
-401	val_401	2008-04-08	12
-421	val_421	2008-04-08	12
-407	val_407	2008-04-08	12
-384	val_384	2008-04-08	12
-256	val_256	2008-04-08	12
-26	val_26	2008-04-08	12
-134	val_134	2008-04-08	12
-67	val_67	2008-04-08	12
-384	val_384	2008-04-08	12
-379	val_379	2008-04-08	12
-18	val_18	2008-04-08	12
-462	val_462	2008-04-08	12
-492	val_492	2008-04-08	12
-100	val_100	2008-04-08	12
-298	val_298	2008-04-08	12
-9	val_9	2008-04-08	12
-341	val_341	2008-04-08	12
-498	val_498	2008-04-08	12
-146	val_146	2008-04-08	12
-458	val_458	2008-04-08	12
-362	val_362	2008-04-08	12
-186	val_186	2008-04-08	12
-285	val_285	2008-04-08	12
-348	val_348	2008-04-08	12
-167	val_167	2008-04-08	12
-18	val_18	2008-04-08	12
-273	val_273	2008-04-08	12
-183	val_183	2008-04-08	12
-281	val_281	2008-04-08	12
-344	val_344	2008-04-08	12
-97	val_97	2008-04-08	12
-469	val_469	2008-04-08	12
-315	val_315	2008-04-08	12
-84	val_84	2008-04-08	12
-28	val_28	2008-04-08	12
-37	val_37	2008-04-08	12
-448	val_448	2008-04-08	12
-152	val_152	2008-04-08	12
-348	val_348	2008-04-08	12
-307	val_307	2008-04-08	12
-194	val_194	2008-04-08	12
-414	val_414	2008-04-08	12
-477	val_477	2008-04-08	12
-222	val_222	2008-04-08	12
-126	val_126	2008-04-08	12
-90	val_90	2008-04-08	12
-169	val_169	2008-04-08	12
-403	val_403	2008-04-08	12
-400	val_400	2008-04-08	12
-200	val_200	2008-04-08	12
-97	val_97	2008-04-08	12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part1-11-8b6be78ae0527e5b7efd7db758966853 b/sql/hive/src/test/resources/golden/load_dyn_part1-11-8b6be78ae0527e5b7efd7db758966853
new file mode 100644
index 0000000000000000000000000000000000000000..ee4601f59e708c4de203abc48fe418b32a3c8797
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/load_dyn_part1-11-8b6be78ae0527e5b7efd7db758966853
@@ -0,0 +1,1000 @@
+0	val_0	2008-12-31	11
+0	val_0	2008-12-31	11
+0	val_0	2008-12-31	11
+2	val_2	2008-12-31	11
+4	val_4	2008-12-31	11
+5	val_5	2008-12-31	11
+5	val_5	2008-12-31	11
+5	val_5	2008-12-31	11
+8	val_8	2008-12-31	11
+9	val_9	2008-12-31	11
+10	val_10	2008-12-31	11
+11	val_11	2008-12-31	11
+12	val_12	2008-12-31	11
+12	val_12	2008-12-31	11
+15	val_15	2008-12-31	11
+15	val_15	2008-12-31	11
+17	val_17	2008-12-31	11
+18	val_18	2008-12-31	11
+18	val_18	2008-12-31	11
+19	val_19	2008-12-31	11
+20	val_20	2008-12-31	11
+24	val_24	2008-12-31	11
+24	val_24	2008-12-31	11
+26	val_26	2008-12-31	11
+26	val_26	2008-12-31	11
+27	val_27	2008-12-31	11
+28	val_28	2008-12-31	11
+30	val_30	2008-12-31	11
+33	val_33	2008-12-31	11
+34	val_34	2008-12-31	11
+35	val_35	2008-12-31	11
+35	val_35	2008-12-31	11
+35	val_35	2008-12-31	11
+37	val_37	2008-12-31	11
+37	val_37	2008-12-31	11
+41	val_41	2008-12-31	11
+42	val_42	2008-12-31	11
+42	val_42	2008-12-31	11
+43	val_43	2008-12-31	11
+44	val_44	2008-12-31	11
+47	val_47	2008-12-31	11
+51	val_51	2008-12-31	11
+51	val_51	2008-12-31	11
+53	val_53	2008-12-31	11
+54	val_54	2008-12-31	11
+57	val_57	2008-12-31	11
+58	val_58	2008-12-31	11
+58	val_58	2008-12-31	11
+64	val_64	2008-12-31	11
+65	val_65	2008-12-31	11
+66	val_66	2008-12-31	11
+67	val_67	2008-12-31	11
+67	val_67	2008-12-31	11
+69	val_69	2008-12-31	11
+70	val_70	2008-12-31	11
+70	val_70	2008-12-31	11
+70	val_70	2008-12-31	11
+72	val_72	2008-12-31	11
+72	val_72	2008-12-31	11
+74	val_74	2008-12-31	11
+76	val_76	2008-12-31	11
+76	val_76	2008-12-31	11
+77	val_77	2008-12-31	11
+78	val_78	2008-12-31	11
+80	val_80	2008-12-31	11
+82	val_82	2008-12-31	11
+83	val_83	2008-12-31	11
+83	val_83	2008-12-31	11
+84	val_84	2008-12-31	11
+84	val_84	2008-12-31	11
+85	val_85	2008-12-31	11
+86	val_86	2008-12-31	11
+87	val_87	2008-12-31	11
+90	val_90	2008-12-31	11
+90	val_90	2008-12-31	11
+90	val_90	2008-12-31	11
+92	val_92	2008-12-31	11
+95	val_95	2008-12-31	11
+95	val_95	2008-12-31	11
+96	val_96	2008-12-31	11
+97	val_97	2008-12-31	11
+97	val_97	2008-12-31	11
+98	val_98	2008-12-31	11
+98	val_98	2008-12-31	11
+100	val_100	2008-12-31	11
+100	val_100	2008-12-31	11
+103	val_103	2008-12-31	11
+103	val_103	2008-12-31	11
+104	val_104	2008-12-31	11
+104	val_104	2008-12-31	11
+105	val_105	2008-12-31	11
+111	val_111	2008-12-31	11
+113	val_113	2008-12-31	11
+113	val_113	2008-12-31	11
+114	val_114	2008-12-31	11
+116	val_116	2008-12-31	11
+118	val_118	2008-12-31	11
+118	val_118	2008-12-31	11
+119	val_119	2008-12-31	11
+119	val_119	2008-12-31	11
+119	val_119	2008-12-31	11
+120	val_120	2008-12-31	11
+120	val_120	2008-12-31	11
+125	val_125	2008-12-31	11
+125	val_125	2008-12-31	11
+126	val_126	2008-12-31	11
+128	val_128	2008-12-31	11
+128	val_128	2008-12-31	11
+128	val_128	2008-12-31	11
+129	val_129	2008-12-31	11
+129	val_129	2008-12-31	11
+131	val_131	2008-12-31	11
+133	val_133	2008-12-31	11
+134	val_134	2008-12-31	11
+134	val_134	2008-12-31	11
+136	val_136	2008-12-31	11
+137	val_137	2008-12-31	11
+137	val_137	2008-12-31	11
+138	val_138	2008-12-31	11
+138	val_138	2008-12-31	11
+138	val_138	2008-12-31	11
+138	val_138	2008-12-31	11
+143	val_143	2008-12-31	11
+145	val_145	2008-12-31	11
+146	val_146	2008-12-31	11
+146	val_146	2008-12-31	11
+149	val_149	2008-12-31	11
+149	val_149	2008-12-31	11
+150	val_150	2008-12-31	11
+152	val_152	2008-12-31	11
+152	val_152	2008-12-31	11
+153	val_153	2008-12-31	11
+155	val_155	2008-12-31	11
+156	val_156	2008-12-31	11
+157	val_157	2008-12-31	11
+158	val_158	2008-12-31	11
+160	val_160	2008-12-31	11
+162	val_162	2008-12-31	11
+163	val_163	2008-12-31	11
+164	val_164	2008-12-31	11
+164	val_164	2008-12-31	11
+165	val_165	2008-12-31	11
+165	val_165	2008-12-31	11
+166	val_166	2008-12-31	11
+167	val_167	2008-12-31	11
+167	val_167	2008-12-31	11
+167	val_167	2008-12-31	11
+168	val_168	2008-12-31	11
+169	val_169	2008-12-31	11
+169	val_169	2008-12-31	11
+169	val_169	2008-12-31	11
+169	val_169	2008-12-31	11
+170	val_170	2008-12-31	11
+172	val_172	2008-12-31	11
+172	val_172	2008-12-31	11
+174	val_174	2008-12-31	11
+174	val_174	2008-12-31	11
+175	val_175	2008-12-31	11
+175	val_175	2008-12-31	11
+176	val_176	2008-12-31	11
+176	val_176	2008-12-31	11
+177	val_177	2008-12-31	11
+178	val_178	2008-12-31	11
+179	val_179	2008-12-31	11
+179	val_179	2008-12-31	11
+180	val_180	2008-12-31	11
+181	val_181	2008-12-31	11
+183	val_183	2008-12-31	11
+186	val_186	2008-12-31	11
+187	val_187	2008-12-31	11
+187	val_187	2008-12-31	11
+187	val_187	2008-12-31	11
+189	val_189	2008-12-31	11
+190	val_190	2008-12-31	11
+191	val_191	2008-12-31	11
+191	val_191	2008-12-31	11
+192	val_192	2008-12-31	11
+193	val_193	2008-12-31	11
+193	val_193	2008-12-31	11
+193	val_193	2008-12-31	11
+194	val_194	2008-12-31	11
+195	val_195	2008-12-31	11
+195	val_195	2008-12-31	11
+196	val_196	2008-12-31	11
+197	val_197	2008-12-31	11
+197	val_197	2008-12-31	11
+199	val_199	2008-12-31	11
+199	val_199	2008-12-31	11
+199	val_199	2008-12-31	11
+200	val_200	2008-12-31	11
+200	val_200	2008-12-31	11
+201	val_201	2008-12-31	11
+202	val_202	2008-12-31	11
+203	val_203	2008-12-31	11
+203	val_203	2008-12-31	11
+205	val_205	2008-12-31	11
+205	val_205	2008-12-31	11
+207	val_207	2008-12-31	11
+207	val_207	2008-12-31	11
+208	val_208	2008-12-31	11
+208	val_208	2008-12-31	11
+208	val_208	2008-12-31	11
+209	val_209	2008-12-31	11
+209	val_209	2008-12-31	11
+213	val_213	2008-12-31	11
+213	val_213	2008-12-31	11
+214	val_214	2008-12-31	11
+216	val_216	2008-12-31	11
+216	val_216	2008-12-31	11
+217	val_217	2008-12-31	11
+217	val_217	2008-12-31	11
+218	val_218	2008-12-31	11
+219	val_219	2008-12-31	11
+219	val_219	2008-12-31	11
+221	val_221	2008-12-31	11
+221	val_221	2008-12-31	11
+222	val_222	2008-12-31	11
+223	val_223	2008-12-31	11
+223	val_223	2008-12-31	11
+224	val_224	2008-12-31	11
+224	val_224	2008-12-31	11
+226	val_226	2008-12-31	11
+228	val_228	2008-12-31	11
+229	val_229	2008-12-31	11
+229	val_229	2008-12-31	11
+230	val_230	2008-12-31	11
+230	val_230	2008-12-31	11
+230	val_230	2008-12-31	11
+230	val_230	2008-12-31	11
+230	val_230	2008-12-31	11
+233	val_233	2008-12-31	11
+233	val_233	2008-12-31	11
+235	val_235	2008-12-31	11
+237	val_237	2008-12-31	11
+237	val_237	2008-12-31	11
+238	val_238	2008-12-31	11
+238	val_238	2008-12-31	11
+239	val_239	2008-12-31	11
+239	val_239	2008-12-31	11
+241	val_241	2008-12-31	11
+242	val_242	2008-12-31	11
+242	val_242	2008-12-31	11
+244	val_244	2008-12-31	11
+247	val_247	2008-12-31	11
+248	val_248	2008-12-31	11
+249	val_249	2008-12-31	11
+252	val_252	2008-12-31	11
+255	val_255	2008-12-31	11
+255	val_255	2008-12-31	11
+256	val_256	2008-12-31	11
+256	val_256	2008-12-31	11
+257	val_257	2008-12-31	11
+258	val_258	2008-12-31	11
+260	val_260	2008-12-31	11
+262	val_262	2008-12-31	11
+263	val_263	2008-12-31	11
+265	val_265	2008-12-31	11
+265	val_265	2008-12-31	11
+266	val_266	2008-12-31	11
+272	val_272	2008-12-31	11
+272	val_272	2008-12-31	11
+273	val_273	2008-12-31	11
+273	val_273	2008-12-31	11
+273	val_273	2008-12-31	11
+274	val_274	2008-12-31	11
+275	val_275	2008-12-31	11
+277	val_277	2008-12-31	11
+277	val_277	2008-12-31	11
+277	val_277	2008-12-31	11
+277	val_277	2008-12-31	11
+278	val_278	2008-12-31	11
+278	val_278	2008-12-31	11
+280	val_280	2008-12-31	11
+280	val_280	2008-12-31	11
+281	val_281	2008-12-31	11
+281	val_281	2008-12-31	11
+282	val_282	2008-12-31	11
+282	val_282	2008-12-31	11
+283	val_283	2008-12-31	11
+284	val_284	2008-12-31	11
+285	val_285	2008-12-31	11
+286	val_286	2008-12-31	11
+287	val_287	2008-12-31	11
+288	val_288	2008-12-31	11
+288	val_288	2008-12-31	11
+289	val_289	2008-12-31	11
+291	val_291	2008-12-31	11
+292	val_292	2008-12-31	11
+296	val_296	2008-12-31	11
+298	val_298	2008-12-31	11
+298	val_298	2008-12-31	11
+298	val_298	2008-12-31	11
+302	val_302	2008-12-31	11
+305	val_305	2008-12-31	11
+306	val_306	2008-12-31	11
+307	val_307	2008-12-31	11
+307	val_307	2008-12-31	11
+308	val_308	2008-12-31	11
+309	val_309	2008-12-31	11
+309	val_309	2008-12-31	11
+310	val_310	2008-12-31	11
+311	val_311	2008-12-31	11
+311	val_311	2008-12-31	11
+311	val_311	2008-12-31	11
+315	val_315	2008-12-31	11
+316	val_316	2008-12-31	11
+316	val_316	2008-12-31	11
+316	val_316	2008-12-31	11
+317	val_317	2008-12-31	11
+317	val_317	2008-12-31	11
+318	val_318	2008-12-31	11
+318	val_318	2008-12-31	11
+318	val_318	2008-12-31	11
+321	val_321	2008-12-31	11
+321	val_321	2008-12-31	11
+322	val_322	2008-12-31	11
+322	val_322	2008-12-31	11
+323	val_323	2008-12-31	11
+325	val_325	2008-12-31	11
+325	val_325	2008-12-31	11
+327	val_327	2008-12-31	11
+327	val_327	2008-12-31	11
+327	val_327	2008-12-31	11
+331	val_331	2008-12-31	11
+331	val_331	2008-12-31	11
+332	val_332	2008-12-31	11
+333	val_333	2008-12-31	11
+333	val_333	2008-12-31	11
+335	val_335	2008-12-31	11
+336	val_336	2008-12-31	11
+338	val_338	2008-12-31	11
+339	val_339	2008-12-31	11
+341	val_341	2008-12-31	11
+342	val_342	2008-12-31	11
+342	val_342	2008-12-31	11
+344	val_344	2008-12-31	11
+344	val_344	2008-12-31	11
+345	val_345	2008-12-31	11
+348	val_348	2008-12-31	11
+348	val_348	2008-12-31	11
+348	val_348	2008-12-31	11
+348	val_348	2008-12-31	11
+348	val_348	2008-12-31	11
+351	val_351	2008-12-31	11
+353	val_353	2008-12-31	11
+353	val_353	2008-12-31	11
+356	val_356	2008-12-31	11
+360	val_360	2008-12-31	11
+362	val_362	2008-12-31	11
+364	val_364	2008-12-31	11
+365	val_365	2008-12-31	11
+366	val_366	2008-12-31	11
+367	val_367	2008-12-31	11
+367	val_367	2008-12-31	11
+368	val_368	2008-12-31	11
+369	val_369	2008-12-31	11
+369	val_369	2008-12-31	11
+369	val_369	2008-12-31	11
+373	val_373	2008-12-31	11
+374	val_374	2008-12-31	11
+375	val_375	2008-12-31	11
+377	val_377	2008-12-31	11
+378	val_378	2008-12-31	11
+379	val_379	2008-12-31	11
+382	val_382	2008-12-31	11
+382	val_382	2008-12-31	11
+384	val_384	2008-12-31	11
+384	val_384	2008-12-31	11
+384	val_384	2008-12-31	11
+386	val_386	2008-12-31	11
+389	val_389	2008-12-31	11
+392	val_392	2008-12-31	11
+393	val_393	2008-12-31	11
+394	val_394	2008-12-31	11
+395	val_395	2008-12-31	11
+395	val_395	2008-12-31	11
+396	val_396	2008-12-31	11
+396	val_396	2008-12-31	11
+396	val_396	2008-12-31	11
+397	val_397	2008-12-31	11
+397	val_397	2008-12-31	11
+399	val_399	2008-12-31	11
+399	val_399	2008-12-31	11
+400	val_400	2008-12-31	11
+401	val_401	2008-12-31	11
+401	val_401	2008-12-31	11
+401	val_401	2008-12-31	11
+401	val_401	2008-12-31	11
+401	val_401	2008-12-31	11
+402	val_402	2008-12-31	11
+403	val_403	2008-12-31	11
+403	val_403	2008-12-31	11
+403	val_403	2008-12-31	11
+404	val_404	2008-12-31	11
+404	val_404	2008-12-31	11
+406	val_406	2008-12-31	11
+406	val_406	2008-12-31	11
+406	val_406	2008-12-31	11
+406	val_406	2008-12-31	11
+407	val_407	2008-12-31	11
+409	val_409	2008-12-31	11
+409	val_409	2008-12-31	11
+409	val_409	2008-12-31	11
+411	val_411	2008-12-31	11
+413	val_413	2008-12-31	11
+413	val_413	2008-12-31	11
+414	val_414	2008-12-31	11
+414	val_414	2008-12-31	11
+417	val_417	2008-12-31	11
+417	val_417	2008-12-31	11
+417	val_417	2008-12-31	11
+418	val_418	2008-12-31	11
+419	val_419	2008-12-31	11
+421	val_421	2008-12-31	11
+424	val_424	2008-12-31	11
+424	val_424	2008-12-31	11
+427	val_427	2008-12-31	11
+429	val_429	2008-12-31	11
+429	val_429	2008-12-31	11
+430	val_430	2008-12-31	11
+430	val_430	2008-12-31	11
+430	val_430	2008-12-31	11
+431	val_431	2008-12-31	11
+431	val_431	2008-12-31	11
+431	val_431	2008-12-31	11
+432	val_432	2008-12-31	11
+435	val_435	2008-12-31	11
+436	val_436	2008-12-31	11
+437	val_437	2008-12-31	11
+438	val_438	2008-12-31	11
+438	val_438	2008-12-31	11
+438	val_438	2008-12-31	11
+439	val_439	2008-12-31	11
+439	val_439	2008-12-31	11
+443	val_443	2008-12-31	11
+444	val_444	2008-12-31	11
+446	val_446	2008-12-31	11
+448	val_448	2008-12-31	11
+449	val_449	2008-12-31	11
+452	val_452	2008-12-31	11
+453	val_453	2008-12-31	11
+454	val_454	2008-12-31	11
+454	val_454	2008-12-31	11
+454	val_454	2008-12-31	11
+455	val_455	2008-12-31	11
+457	val_457	2008-12-31	11
+458	val_458	2008-12-31	11
+458	val_458	2008-12-31	11
+459	val_459	2008-12-31	11
+459	val_459	2008-12-31	11
+460	val_460	2008-12-31	11
+462	val_462	2008-12-31	11
+462	val_462	2008-12-31	11
+463	val_463	2008-12-31	11
+463	val_463	2008-12-31	11
+466	val_466	2008-12-31	11
+466	val_466	2008-12-31	11
+466	val_466	2008-12-31	11
+467	val_467	2008-12-31	11
+468	val_468	2008-12-31	11
+468	val_468	2008-12-31	11
+468	val_468	2008-12-31	11
+468	val_468	2008-12-31	11
+469	val_469	2008-12-31	11
+469	val_469	2008-12-31	11
+469	val_469	2008-12-31	11
+469	val_469	2008-12-31	11
+469	val_469	2008-12-31	11
+470	val_470	2008-12-31	11
+472	val_472	2008-12-31	11
+475	val_475	2008-12-31	11
+477	val_477	2008-12-31	11
+478	val_478	2008-12-31	11
+478	val_478	2008-12-31	11
+479	val_479	2008-12-31	11
+480	val_480	2008-12-31	11
+480	val_480	2008-12-31	11
+480	val_480	2008-12-31	11
+481	val_481	2008-12-31	11
+482	val_482	2008-12-31	11
+483	val_483	2008-12-31	11
+484	val_484	2008-12-31	11
+485	val_485	2008-12-31	11
+487	val_487	2008-12-31	11
+489	val_489	2008-12-31	11
+489	val_489	2008-12-31	11
+489	val_489	2008-12-31	11
+489	val_489	2008-12-31	11
+490	val_490	2008-12-31	11
+491	val_491	2008-12-31	11
+492	val_492	2008-12-31	11
+492	val_492	2008-12-31	11
+493	val_493	2008-12-31	11
+494	val_494	2008-12-31	11
+495	val_495	2008-12-31	11
+496	val_496	2008-12-31	11
+497	val_497	2008-12-31	11
+498	val_498	2008-12-31	11
+498	val_498	2008-12-31	11
+498	val_498	2008-12-31	11
+0	val_0	2008-12-31	12
+0	val_0	2008-12-31	12
+0	val_0	2008-12-31	12
+2	val_2	2008-12-31	12
+4	val_4	2008-12-31	12
+5	val_5	2008-12-31	12
+5	val_5	2008-12-31	12
+5	val_5	2008-12-31	12
+8	val_8	2008-12-31	12
+9	val_9	2008-12-31	12
+10	val_10	2008-12-31	12
+11	val_11	2008-12-31	12
+12	val_12	2008-12-31	12
+12	val_12	2008-12-31	12
+15	val_15	2008-12-31	12
+15	val_15	2008-12-31	12
+17	val_17	2008-12-31	12
+18	val_18	2008-12-31	12
+18	val_18	2008-12-31	12
+19	val_19	2008-12-31	12
+20	val_20	2008-12-31	12
+24	val_24	2008-12-31	12
+24	val_24	2008-12-31	12
+26	val_26	2008-12-31	12
+26	val_26	2008-12-31	12
+27	val_27	2008-12-31	12
+28	val_28	2008-12-31	12
+30	val_30	2008-12-31	12
+33	val_33	2008-12-31	12
+34	val_34	2008-12-31	12
+35	val_35	2008-12-31	12
+35	val_35	2008-12-31	12
+35	val_35	2008-12-31	12
+37	val_37	2008-12-31	12
+37	val_37	2008-12-31	12
+41	val_41	2008-12-31	12
+42	val_42	2008-12-31	12
+42	val_42	2008-12-31	12
+43	val_43	2008-12-31	12
+44	val_44	2008-12-31	12
+47	val_47	2008-12-31	12
+51	val_51	2008-12-31	12
+51	val_51	2008-12-31	12
+53	val_53	2008-12-31	12
+54	val_54	2008-12-31	12
+57	val_57	2008-12-31	12
+58	val_58	2008-12-31	12
+58	val_58	2008-12-31	12
+64	val_64	2008-12-31	12
+65	val_65	2008-12-31	12
+66	val_66	2008-12-31	12
+67	val_67	2008-12-31	12
+67	val_67	2008-12-31	12
+69	val_69	2008-12-31	12
+70	val_70	2008-12-31	12
+70	val_70	2008-12-31	12
+70	val_70	2008-12-31	12
+72	val_72	2008-12-31	12
+72	val_72	2008-12-31	12
+74	val_74	2008-12-31	12
+76	val_76	2008-12-31	12
+76	val_76	2008-12-31	12
+77	val_77	2008-12-31	12
+78	val_78	2008-12-31	12
+80	val_80	2008-12-31	12
+82	val_82	2008-12-31	12
+83	val_83	2008-12-31	12
+83	val_83	2008-12-31	12
+84	val_84	2008-12-31	12
+84	val_84	2008-12-31	12
+85	val_85	2008-12-31	12
+86	val_86	2008-12-31	12
+87	val_87	2008-12-31	12
+90	val_90	2008-12-31	12
+90	val_90	2008-12-31	12
+90	val_90	2008-12-31	12
+92	val_92	2008-12-31	12
+95	val_95	2008-12-31	12
+95	val_95	2008-12-31	12
+96	val_96	2008-12-31	12
+97	val_97	2008-12-31	12
+97	val_97	2008-12-31	12
+98	val_98	2008-12-31	12
+98	val_98	2008-12-31	12
+100	val_100	2008-12-31	12
+100	val_100	2008-12-31	12
+103	val_103	2008-12-31	12
+103	val_103	2008-12-31	12
+104	val_104	2008-12-31	12
+104	val_104	2008-12-31	12
+105	val_105	2008-12-31	12
+111	val_111	2008-12-31	12
+113	val_113	2008-12-31	12
+113	val_113	2008-12-31	12
+114	val_114	2008-12-31	12
+116	val_116	2008-12-31	12
+118	val_118	2008-12-31	12
+118	val_118	2008-12-31	12
+119	val_119	2008-12-31	12
+119	val_119	2008-12-31	12
+119	val_119	2008-12-31	12
+120	val_120	2008-12-31	12
+120	val_120	2008-12-31	12
+125	val_125	2008-12-31	12
+125	val_125	2008-12-31	12
+126	val_126	2008-12-31	12
+128	val_128	2008-12-31	12
+128	val_128	2008-12-31	12
+128	val_128	2008-12-31	12
+129	val_129	2008-12-31	12
+129	val_129	2008-12-31	12
+131	val_131	2008-12-31	12
+133	val_133	2008-12-31	12
+134	val_134	2008-12-31	12
+134	val_134	2008-12-31	12
+136	val_136	2008-12-31	12
+137	val_137	2008-12-31	12
+137	val_137	2008-12-31	12
+138	val_138	2008-12-31	12
+138	val_138	2008-12-31	12
+138	val_138	2008-12-31	12
+138	val_138	2008-12-31	12
+143	val_143	2008-12-31	12
+145	val_145	2008-12-31	12
+146	val_146	2008-12-31	12
+146	val_146	2008-12-31	12
+149	val_149	2008-12-31	12
+149	val_149	2008-12-31	12
+150	val_150	2008-12-31	12
+152	val_152	2008-12-31	12
+152	val_152	2008-12-31	12
+153	val_153	2008-12-31	12
+155	val_155	2008-12-31	12
+156	val_156	2008-12-31	12
+157	val_157	2008-12-31	12
+158	val_158	2008-12-31	12
+160	val_160	2008-12-31	12
+162	val_162	2008-12-31	12
+163	val_163	2008-12-31	12
+164	val_164	2008-12-31	12
+164	val_164	2008-12-31	12
+165	val_165	2008-12-31	12
+165	val_165	2008-12-31	12
+166	val_166	2008-12-31	12
+167	val_167	2008-12-31	12
+167	val_167	2008-12-31	12
+167	val_167	2008-12-31	12
+168	val_168	2008-12-31	12
+169	val_169	2008-12-31	12
+169	val_169	2008-12-31	12
+169	val_169	2008-12-31	12
+169	val_169	2008-12-31	12
+170	val_170	2008-12-31	12
+172	val_172	2008-12-31	12
+172	val_172	2008-12-31	12
+174	val_174	2008-12-31	12
+174	val_174	2008-12-31	12
+175	val_175	2008-12-31	12
+175	val_175	2008-12-31	12
+176	val_176	2008-12-31	12
+176	val_176	2008-12-31	12
+177	val_177	2008-12-31	12
+178	val_178	2008-12-31	12
+179	val_179	2008-12-31	12
+179	val_179	2008-12-31	12
+180	val_180	2008-12-31	12
+181	val_181	2008-12-31	12
+183	val_183	2008-12-31	12
+186	val_186	2008-12-31	12
+187	val_187	2008-12-31	12
+187	val_187	2008-12-31	12
+187	val_187	2008-12-31	12
+189	val_189	2008-12-31	12
+190	val_190	2008-12-31	12
+191	val_191	2008-12-31	12
+191	val_191	2008-12-31	12
+192	val_192	2008-12-31	12
+193	val_193	2008-12-31	12
+193	val_193	2008-12-31	12
+193	val_193	2008-12-31	12
+194	val_194	2008-12-31	12
+195	val_195	2008-12-31	12
+195	val_195	2008-12-31	12
+196	val_196	2008-12-31	12
+197	val_197	2008-12-31	12
+197	val_197	2008-12-31	12
+199	val_199	2008-12-31	12
+199	val_199	2008-12-31	12
+199	val_199	2008-12-31	12
+200	val_200	2008-12-31	12
+200	val_200	2008-12-31	12
+201	val_201	2008-12-31	12
+202	val_202	2008-12-31	12
+203	val_203	2008-12-31	12
+203	val_203	2008-12-31	12
+205	val_205	2008-12-31	12
+205	val_205	2008-12-31	12
+207	val_207	2008-12-31	12
+207	val_207	2008-12-31	12
+208	val_208	2008-12-31	12
+208	val_208	2008-12-31	12
+208	val_208	2008-12-31	12
+209	val_209	2008-12-31	12
+209	val_209	2008-12-31	12
+213	val_213	2008-12-31	12
+213	val_213	2008-12-31	12
+214	val_214	2008-12-31	12
+216	val_216	2008-12-31	12
+216	val_216	2008-12-31	12
+217	val_217	2008-12-31	12
+217	val_217	2008-12-31	12
+218	val_218	2008-12-31	12
+219	val_219	2008-12-31	12
+219	val_219	2008-12-31	12
+221	val_221	2008-12-31	12
+221	val_221	2008-12-31	12
+222	val_222	2008-12-31	12
+223	val_223	2008-12-31	12
+223	val_223	2008-12-31	12
+224	val_224	2008-12-31	12
+224	val_224	2008-12-31	12
+226	val_226	2008-12-31	12
+228	val_228	2008-12-31	12
+229	val_229	2008-12-31	12
+229	val_229	2008-12-31	12
+230	val_230	2008-12-31	12
+230	val_230	2008-12-31	12
+230	val_230	2008-12-31	12
+230	val_230	2008-12-31	12
+230	val_230	2008-12-31	12
+233	val_233	2008-12-31	12
+233	val_233	2008-12-31	12
+235	val_235	2008-12-31	12
+237	val_237	2008-12-31	12
+237	val_237	2008-12-31	12
+238	val_238	2008-12-31	12
+238	val_238	2008-12-31	12
+239	val_239	2008-12-31	12
+239	val_239	2008-12-31	12
+241	val_241	2008-12-31	12
+242	val_242	2008-12-31	12
+242	val_242	2008-12-31	12
+244	val_244	2008-12-31	12
+247	val_247	2008-12-31	12
+248	val_248	2008-12-31	12
+249	val_249	2008-12-31	12
+252	val_252	2008-12-31	12
+255	val_255	2008-12-31	12
+255	val_255	2008-12-31	12
+256	val_256	2008-12-31	12
+256	val_256	2008-12-31	12
+257	val_257	2008-12-31	12
+258	val_258	2008-12-31	12
+260	val_260	2008-12-31	12
+262	val_262	2008-12-31	12
+263	val_263	2008-12-31	12
+265	val_265	2008-12-31	12
+265	val_265	2008-12-31	12
+266	val_266	2008-12-31	12
+272	val_272	2008-12-31	12
+272	val_272	2008-12-31	12
+273	val_273	2008-12-31	12
+273	val_273	2008-12-31	12
+273	val_273	2008-12-31	12
+274	val_274	2008-12-31	12
+275	val_275	2008-12-31	12
+277	val_277	2008-12-31	12
+277	val_277	2008-12-31	12
+277	val_277	2008-12-31	12
+277	val_277	2008-12-31	12
+278	val_278	2008-12-31	12
+278	val_278	2008-12-31	12
+280	val_280	2008-12-31	12
+280	val_280	2008-12-31	12
+281	val_281	2008-12-31	12
+281	val_281	2008-12-31	12
+282	val_282	2008-12-31	12
+282	val_282	2008-12-31	12
+283	val_283	2008-12-31	12
+284	val_284	2008-12-31	12
+285	val_285	2008-12-31	12
+286	val_286	2008-12-31	12
+287	val_287	2008-12-31	12
+288	val_288	2008-12-31	12
+288	val_288	2008-12-31	12
+289	val_289	2008-12-31	12
+291	val_291	2008-12-31	12
+292	val_292	2008-12-31	12
+296	val_296	2008-12-31	12
+298	val_298	2008-12-31	12
+298	val_298	2008-12-31	12
+298	val_298	2008-12-31	12
+302	val_302	2008-12-31	12
+305	val_305	2008-12-31	12
+306	val_306	2008-12-31	12
+307	val_307	2008-12-31	12
+307	val_307	2008-12-31	12
+308	val_308	2008-12-31	12
+309	val_309	2008-12-31	12
+309	val_309	2008-12-31	12
+310	val_310	2008-12-31	12
+311	val_311	2008-12-31	12
+311	val_311	2008-12-31	12
+311	val_311	2008-12-31	12
+315	val_315	2008-12-31	12
+316	val_316	2008-12-31	12
+316	val_316	2008-12-31	12
+316	val_316	2008-12-31	12
+317	val_317	2008-12-31	12
+317	val_317	2008-12-31	12
+318	val_318	2008-12-31	12
+318	val_318	2008-12-31	12
+318	val_318	2008-12-31	12
+321	val_321	2008-12-31	12
+321	val_321	2008-12-31	12
+322	val_322	2008-12-31	12
+322	val_322	2008-12-31	12
+323	val_323	2008-12-31	12
+325	val_325	2008-12-31	12
+325	val_325	2008-12-31	12
+327	val_327	2008-12-31	12
+327	val_327	2008-12-31	12
+327	val_327	2008-12-31	12
+331	val_331	2008-12-31	12
+331	val_331	2008-12-31	12
+332	val_332	2008-12-31	12
+333	val_333	2008-12-31	12
+333	val_333	2008-12-31	12
+335	val_335	2008-12-31	12
+336	val_336	2008-12-31	12
+338	val_338	2008-12-31	12
+339	val_339	2008-12-31	12
+341	val_341	2008-12-31	12
+342	val_342	2008-12-31	12
+342	val_342	2008-12-31	12
+344	val_344	2008-12-31	12
+344	val_344	2008-12-31	12
+345	val_345	2008-12-31	12
+348	val_348	2008-12-31	12
+348	val_348	2008-12-31	12
+348	val_348	2008-12-31	12
+348	val_348	2008-12-31	12
+348	val_348	2008-12-31	12
+351	val_351	2008-12-31	12
+353	val_353	2008-12-31	12
+353	val_353	2008-12-31	12
+356	val_356	2008-12-31	12
+360	val_360	2008-12-31	12
+362	val_362	2008-12-31	12
+364	val_364	2008-12-31	12
+365	val_365	2008-12-31	12
+366	val_366	2008-12-31	12
+367	val_367	2008-12-31	12
+367	val_367	2008-12-31	12
+368	val_368	2008-12-31	12
+369	val_369	2008-12-31	12
+369	val_369	2008-12-31	12
+369	val_369	2008-12-31	12
+373	val_373	2008-12-31	12
+374	val_374	2008-12-31	12
+375	val_375	2008-12-31	12
+377	val_377	2008-12-31	12
+378	val_378	2008-12-31	12
+379	val_379	2008-12-31	12
+382	val_382	2008-12-31	12
+382	val_382	2008-12-31	12
+384	val_384	2008-12-31	12
+384	val_384	2008-12-31	12
+384	val_384	2008-12-31	12
+386	val_386	2008-12-31	12
+389	val_389	2008-12-31	12
+392	val_392	2008-12-31	12
+393	val_393	2008-12-31	12
+394	val_394	2008-12-31	12
+395	val_395	2008-12-31	12
+395	val_395	2008-12-31	12
+396	val_396	2008-12-31	12
+396	val_396	2008-12-31	12
+396	val_396	2008-12-31	12
+397	val_397	2008-12-31	12
+397	val_397	2008-12-31	12
+399	val_399	2008-12-31	12
+399	val_399	2008-12-31	12
+400	val_400	2008-12-31	12
+401	val_401	2008-12-31	12
+401	val_401	2008-12-31	12
+401	val_401	2008-12-31	12
+401	val_401	2008-12-31	12
+401	val_401	2008-12-31	12
+402	val_402	2008-12-31	12
+403	val_403	2008-12-31	12
+403	val_403	2008-12-31	12
+403	val_403	2008-12-31	12
+404	val_404	2008-12-31	12
+404	val_404	2008-12-31	12
+406	val_406	2008-12-31	12
+406	val_406	2008-12-31	12
+406	val_406	2008-12-31	12
+406	val_406	2008-12-31	12
+407	val_407	2008-12-31	12
+409	val_409	2008-12-31	12
+409	val_409	2008-12-31	12
+409	val_409	2008-12-31	12
+411	val_411	2008-12-31	12
+413	val_413	2008-12-31	12
+413	val_413	2008-12-31	12
+414	val_414	2008-12-31	12
+414	val_414	2008-12-31	12
+417	val_417	2008-12-31	12
+417	val_417	2008-12-31	12
+417	val_417	2008-12-31	12
+418	val_418	2008-12-31	12
+419	val_419	2008-12-31	12
+421	val_421	2008-12-31	12
+424	val_424	2008-12-31	12
+424	val_424	2008-12-31	12
+427	val_427	2008-12-31	12
+429	val_429	2008-12-31	12
+429	val_429	2008-12-31	12
+430	val_430	2008-12-31	12
+430	val_430	2008-12-31	12
+430	val_430	2008-12-31	12
+431	val_431	2008-12-31	12
+431	val_431	2008-12-31	12
+431	val_431	2008-12-31	12
+432	val_432	2008-12-31	12
+435	val_435	2008-12-31	12
+436	val_436	2008-12-31	12
+437	val_437	2008-12-31	12
+438	val_438	2008-12-31	12
+438	val_438	2008-12-31	12
+438	val_438	2008-12-31	12
+439	val_439	2008-12-31	12
+439	val_439	2008-12-31	12
+443	val_443	2008-12-31	12
+444	val_444	2008-12-31	12
+446	val_446	2008-12-31	12
+448	val_448	2008-12-31	12
+449	val_449	2008-12-31	12
+452	val_452	2008-12-31	12
+453	val_453	2008-12-31	12
+454	val_454	2008-12-31	12
+454	val_454	2008-12-31	12
+454	val_454	2008-12-31	12
+455	val_455	2008-12-31	12
+457	val_457	2008-12-31	12
+458	val_458	2008-12-31	12
+458	val_458	2008-12-31	12
+459	val_459	2008-12-31	12
+459	val_459	2008-12-31	12
+460	val_460	2008-12-31	12
+462	val_462	2008-12-31	12
+462	val_462	2008-12-31	12
+463	val_463	2008-12-31	12
+463	val_463	2008-12-31	12
+466	val_466	2008-12-31	12
+466	val_466	2008-12-31	12
+466	val_466	2008-12-31	12
+467	val_467	2008-12-31	12
+468	val_468	2008-12-31	12
+468	val_468	2008-12-31	12
+468	val_468	2008-12-31	12
+468	val_468	2008-12-31	12
+469	val_469	2008-12-31	12
+469	val_469	2008-12-31	12
+469	val_469	2008-12-31	12
+469	val_469	2008-12-31	12
+469	val_469	2008-12-31	12
+470	val_470	2008-12-31	12
+472	val_472	2008-12-31	12
+475	val_475	2008-12-31	12
+477	val_477	2008-12-31	12
+478	val_478	2008-12-31	12
+478	val_478	2008-12-31	12
+479	val_479	2008-12-31	12
+480	val_480	2008-12-31	12
+480	val_480	2008-12-31	12
+480	val_480	2008-12-31	12
+481	val_481	2008-12-31	12
+482	val_482	2008-12-31	12
+483	val_483	2008-12-31	12
+484	val_484	2008-12-31	12
+485	val_485	2008-12-31	12
+487	val_487	2008-12-31	12
+489	val_489	2008-12-31	12
+489	val_489	2008-12-31	12
+489	val_489	2008-12-31	12
+489	val_489	2008-12-31	12
+490	val_490	2008-12-31	12
+491	val_491	2008-12-31	12
+492	val_492	2008-12-31	12
+492	val_492	2008-12-31	12
+493	val_493	2008-12-31	12
+494	val_494	2008-12-31	12
+495	val_495	2008-12-31	12
+496	val_496	2008-12-31	12
+497	val_497	2008-12-31	12
+498	val_498	2008-12-31	12
+498	val_498	2008-12-31	12
+498	val_498	2008-12-31	12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 b/sql/hive/src/test/resources/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798
deleted file mode 100644
index 653516475da222e533bd2651cda6fdb1cdc1576a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798
+++ /dev/null
@@ -1,1000 +0,0 @@
-238	val_238	2008-12-31	11
-86	val_86	2008-12-31	11
-311	val_311	2008-12-31	11
-27	val_27	2008-12-31	11
-165	val_165	2008-12-31	11
-409	val_409	2008-12-31	11
-255	val_255	2008-12-31	11
-278	val_278	2008-12-31	11
-98	val_98	2008-12-31	11
-484	val_484	2008-12-31	11
-265	val_265	2008-12-31	11
-193	val_193	2008-12-31	11
-401	val_401	2008-12-31	11
-150	val_150	2008-12-31	11
-273	val_273	2008-12-31	11
-224	val_224	2008-12-31	11
-369	val_369	2008-12-31	11
-66	val_66	2008-12-31	11
-128	val_128	2008-12-31	11
-213	val_213	2008-12-31	11
-146	val_146	2008-12-31	11
-406	val_406	2008-12-31	11
-429	val_429	2008-12-31	11
-374	val_374	2008-12-31	11
-152	val_152	2008-12-31	11
-469	val_469	2008-12-31	11
-145	val_145	2008-12-31	11
-495	val_495	2008-12-31	11
-37	val_37	2008-12-31	11
-327	val_327	2008-12-31	11
-281	val_281	2008-12-31	11
-277	val_277	2008-12-31	11
-209	val_209	2008-12-31	11
-15	val_15	2008-12-31	11
-82	val_82	2008-12-31	11
-403	val_403	2008-12-31	11
-166	val_166	2008-12-31	11
-417	val_417	2008-12-31	11
-430	val_430	2008-12-31	11
-252	val_252	2008-12-31	11
-292	val_292	2008-12-31	11
-219	val_219	2008-12-31	11
-287	val_287	2008-12-31	11
-153	val_153	2008-12-31	11
-193	val_193	2008-12-31	11
-338	val_338	2008-12-31	11
-446	val_446	2008-12-31	11
-459	val_459	2008-12-31	11
-394	val_394	2008-12-31	11
-237	val_237	2008-12-31	11
-482	val_482	2008-12-31	11
-174	val_174	2008-12-31	11
-413	val_413	2008-12-31	11
-494	val_494	2008-12-31	11
-207	val_207	2008-12-31	11
-199	val_199	2008-12-31	11
-466	val_466	2008-12-31	11
-208	val_208	2008-12-31	11
-174	val_174	2008-12-31	11
-399	val_399	2008-12-31	11
-396	val_396	2008-12-31	11
-247	val_247	2008-12-31	11
-417	val_417	2008-12-31	11
-489	val_489	2008-12-31	11
-162	val_162	2008-12-31	11
-377	val_377	2008-12-31	11
-397	val_397	2008-12-31	11
-309	val_309	2008-12-31	11
-365	val_365	2008-12-31	11
-266	val_266	2008-12-31	11
-439	val_439	2008-12-31	11
-342	val_342	2008-12-31	11
-367	val_367	2008-12-31	11
-325	val_325	2008-12-31	11
-167	val_167	2008-12-31	11
-195	val_195	2008-12-31	11
-475	val_475	2008-12-31	11
-17	val_17	2008-12-31	11
-113	val_113	2008-12-31	11
-155	val_155	2008-12-31	11
-203	val_203	2008-12-31	11
-339	val_339	2008-12-31	11
-0	val_0	2008-12-31	11
-455	val_455	2008-12-31	11
-128	val_128	2008-12-31	11
-311	val_311	2008-12-31	11
-316	val_316	2008-12-31	11
-57	val_57	2008-12-31	11
-302	val_302	2008-12-31	11
-205	val_205	2008-12-31	11
-149	val_149	2008-12-31	11
-438	val_438	2008-12-31	11
-345	val_345	2008-12-31	11
-129	val_129	2008-12-31	11
-170	val_170	2008-12-31	11
-20	val_20	2008-12-31	11
-489	val_489	2008-12-31	11
-157	val_157	2008-12-31	11
-378	val_378	2008-12-31	11
-221	val_221	2008-12-31	11
-92	val_92	2008-12-31	11
-111	val_111	2008-12-31	11
-47	val_47	2008-12-31	11
-72	val_72	2008-12-31	11
-4	val_4	2008-12-31	11
-280	val_280	2008-12-31	11
-35	val_35	2008-12-31	11
-427	val_427	2008-12-31	11
-277	val_277	2008-12-31	11
-208	val_208	2008-12-31	11
-356	val_356	2008-12-31	11
-399	val_399	2008-12-31	11
-169	val_169	2008-12-31	11
-382	val_382	2008-12-31	11
-498	val_498	2008-12-31	11
-125	val_125	2008-12-31	11
-386	val_386	2008-12-31	11
-437	val_437	2008-12-31	11
-469	val_469	2008-12-31	11
-192	val_192	2008-12-31	11
-286	val_286	2008-12-31	11
-187	val_187	2008-12-31	11
-176	val_176	2008-12-31	11
-54	val_54	2008-12-31	11
-459	val_459	2008-12-31	11
-51	val_51	2008-12-31	11
-138	val_138	2008-12-31	11
-103	val_103	2008-12-31	11
-239	val_239	2008-12-31	11
-213	val_213	2008-12-31	11
-216	val_216	2008-12-31	11
-430	val_430	2008-12-31	11
-278	val_278	2008-12-31	11
-176	val_176	2008-12-31	11
-289	val_289	2008-12-31	11
-221	val_221	2008-12-31	11
-65	val_65	2008-12-31	11
-318	val_318	2008-12-31	11
-332	val_332	2008-12-31	11
-311	val_311	2008-12-31	11
-275	val_275	2008-12-31	11
-137	val_137	2008-12-31	11
-241	val_241	2008-12-31	11
-83	val_83	2008-12-31	11
-333	val_333	2008-12-31	11
-180	val_180	2008-12-31	11
-284	val_284	2008-12-31	11
-12	val_12	2008-12-31	11
-230	val_230	2008-12-31	11
-181	val_181	2008-12-31	11
-67	val_67	2008-12-31	11
-260	val_260	2008-12-31	11
-404	val_404	2008-12-31	11
-384	val_384	2008-12-31	11
-489	val_489	2008-12-31	11
-353	val_353	2008-12-31	11
-373	val_373	2008-12-31	11
-272	val_272	2008-12-31	11
-138	val_138	2008-12-31	11
-217	val_217	2008-12-31	11
-84	val_84	2008-12-31	11
-348	val_348	2008-12-31	11
-466	val_466	2008-12-31	11
-58	val_58	2008-12-31	11
-8	val_8	2008-12-31	11
-411	val_411	2008-12-31	11
-230	val_230	2008-12-31	11
-208	val_208	2008-12-31	11
-348	val_348	2008-12-31	11
-24	val_24	2008-12-31	11
-463	val_463	2008-12-31	11
-431	val_431	2008-12-31	11
-179	val_179	2008-12-31	11
-172	val_172	2008-12-31	11
-42	val_42	2008-12-31	11
-129	val_129	2008-12-31	11
-158	val_158	2008-12-31	11
-119	val_119	2008-12-31	11
-496	val_496	2008-12-31	11
-0	val_0	2008-12-31	11
-322	val_322	2008-12-31	11
-197	val_197	2008-12-31	11
-468	val_468	2008-12-31	11
-393	val_393	2008-12-31	11
-454	val_454	2008-12-31	11
-100	val_100	2008-12-31	11
-298	val_298	2008-12-31	11
-199	val_199	2008-12-31	11
-191	val_191	2008-12-31	11
-418	val_418	2008-12-31	11
-96	val_96	2008-12-31	11
-26	val_26	2008-12-31	11
-165	val_165	2008-12-31	11
-327	val_327	2008-12-31	11
-230	val_230	2008-12-31	11
-205	val_205	2008-12-31	11
-120	val_120	2008-12-31	11
-131	val_131	2008-12-31	11
-51	val_51	2008-12-31	11
-404	val_404	2008-12-31	11
-43	val_43	2008-12-31	11
-436	val_436	2008-12-31	11
-156	val_156	2008-12-31	11
-469	val_469	2008-12-31	11
-468	val_468	2008-12-31	11
-308	val_308	2008-12-31	11
-95	val_95	2008-12-31	11
-196	val_196	2008-12-31	11
-288	val_288	2008-12-31	11
-481	val_481	2008-12-31	11
-457	val_457	2008-12-31	11
-98	val_98	2008-12-31	11
-282	val_282	2008-12-31	11
-197	val_197	2008-12-31	11
-187	val_187	2008-12-31	11
-318	val_318	2008-12-31	11
-318	val_318	2008-12-31	11
-409	val_409	2008-12-31	11
-470	val_470	2008-12-31	11
-137	val_137	2008-12-31	11
-369	val_369	2008-12-31	11
-316	val_316	2008-12-31	11
-169	val_169	2008-12-31	11
-413	val_413	2008-12-31	11
-85	val_85	2008-12-31	11
-77	val_77	2008-12-31	11
-0	val_0	2008-12-31	11
-490	val_490	2008-12-31	11
-87	val_87	2008-12-31	11
-364	val_364	2008-12-31	11
-179	val_179	2008-12-31	11
-118	val_118	2008-12-31	11
-134	val_134	2008-12-31	11
-395	val_395	2008-12-31	11
-282	val_282	2008-12-31	11
-138	val_138	2008-12-31	11
-238	val_238	2008-12-31	11
-419	val_419	2008-12-31	11
-15	val_15	2008-12-31	11
-118	val_118	2008-12-31	11
-72	val_72	2008-12-31	11
-90	val_90	2008-12-31	11
-307	val_307	2008-12-31	11
-19	val_19	2008-12-31	11
-435	val_435	2008-12-31	11
-10	val_10	2008-12-31	11
-277	val_277	2008-12-31	11
-273	val_273	2008-12-31	11
-306	val_306	2008-12-31	11
-224	val_224	2008-12-31	11
-309	val_309	2008-12-31	11
-389	val_389	2008-12-31	11
-327	val_327	2008-12-31	11
-242	val_242	2008-12-31	11
-369	val_369	2008-12-31	11
-392	val_392	2008-12-31	11
-272	val_272	2008-12-31	11
-331	val_331	2008-12-31	11
-401	val_401	2008-12-31	11
-242	val_242	2008-12-31	11
-452	val_452	2008-12-31	11
-177	val_177	2008-12-31	11
-226	val_226	2008-12-31	11
-5	val_5	2008-12-31	11
-497	val_497	2008-12-31	11
-402	val_402	2008-12-31	11
-396	val_396	2008-12-31	11
-317	val_317	2008-12-31	11
-395	val_395	2008-12-31	11
-58	val_58	2008-12-31	11
-35	val_35	2008-12-31	11
-336	val_336	2008-12-31	11
-95	val_95	2008-12-31	11
-11	val_11	2008-12-31	11
-168	val_168	2008-12-31	11
-34	val_34	2008-12-31	11
-229	val_229	2008-12-31	11
-233	val_233	2008-12-31	11
-143	val_143	2008-12-31	11
-472	val_472	2008-12-31	11
-322	val_322	2008-12-31	11
-498	val_498	2008-12-31	11
-160	val_160	2008-12-31	11
-195	val_195	2008-12-31	11
-42	val_42	2008-12-31	11
-321	val_321	2008-12-31	11
-430	val_430	2008-12-31	11
-119	val_119	2008-12-31	11
-489	val_489	2008-12-31	11
-458	val_458	2008-12-31	11
-78	val_78	2008-12-31	11
-76	val_76	2008-12-31	11
-41	val_41	2008-12-31	11
-223	val_223	2008-12-31	11
-492	val_492	2008-12-31	11
-149	val_149	2008-12-31	11
-449	val_449	2008-12-31	11
-218	val_218	2008-12-31	11
-228	val_228	2008-12-31	11
-138	val_138	2008-12-31	11
-453	val_453	2008-12-31	11
-30	val_30	2008-12-31	11
-209	val_209	2008-12-31	11
-64	val_64	2008-12-31	11
-468	val_468	2008-12-31	11
-76	val_76	2008-12-31	11
-74	val_74	2008-12-31	11
-342	val_342	2008-12-31	11
-69	val_69	2008-12-31	11
-230	val_230	2008-12-31	11
-33	val_33	2008-12-31	11
-368	val_368	2008-12-31	11
-103	val_103	2008-12-31	11
-296	val_296	2008-12-31	11
-113	val_113	2008-12-31	11
-216	val_216	2008-12-31	11
-367	val_367	2008-12-31	11
-344	val_344	2008-12-31	11
-167	val_167	2008-12-31	11
-274	val_274	2008-12-31	11
-219	val_219	2008-12-31	11
-239	val_239	2008-12-31	11
-485	val_485	2008-12-31	11
-116	val_116	2008-12-31	11
-223	val_223	2008-12-31	11
-256	val_256	2008-12-31	11
-263	val_263	2008-12-31	11
-70	val_70	2008-12-31	11
-487	val_487	2008-12-31	11
-480	val_480	2008-12-31	11
-401	val_401	2008-12-31	11
-288	val_288	2008-12-31	11
-191	val_191	2008-12-31	11
-5	val_5	2008-12-31	11
-244	val_244	2008-12-31	11
-438	val_438	2008-12-31	11
-128	val_128	2008-12-31	11
-467	val_467	2008-12-31	11
-432	val_432	2008-12-31	11
-202	val_202	2008-12-31	11
-316	val_316	2008-12-31	11
-229	val_229	2008-12-31	11
-469	val_469	2008-12-31	11
-463	val_463	2008-12-31	11
-280	val_280	2008-12-31	11
-2	val_2	2008-12-31	11
-35	val_35	2008-12-31	11
-283	val_283	2008-12-31	11
-331	val_331	2008-12-31	11
-235	val_235	2008-12-31	11
-80	val_80	2008-12-31	11
-44	val_44	2008-12-31	11
-193	val_193	2008-12-31	11
-321	val_321	2008-12-31	11
-335	val_335	2008-12-31	11
-104	val_104	2008-12-31	11
-466	val_466	2008-12-31	11
-366	val_366	2008-12-31	11
-175	val_175	2008-12-31	11
-403	val_403	2008-12-31	11
-483	val_483	2008-12-31	11
-53	val_53	2008-12-31	11
-105	val_105	2008-12-31	11
-257	val_257	2008-12-31	11
-406	val_406	2008-12-31	11
-409	val_409	2008-12-31	11
-190	val_190	2008-12-31	11
-406	val_406	2008-12-31	11
-401	val_401	2008-12-31	11
-114	val_114	2008-12-31	11
-258	val_258	2008-12-31	11
-90	val_90	2008-12-31	11
-203	val_203	2008-12-31	11
-262	val_262	2008-12-31	11
-348	val_348	2008-12-31	11
-424	val_424	2008-12-31	11
-12	val_12	2008-12-31	11
-396	val_396	2008-12-31	11
-201	val_201	2008-12-31	11
-217	val_217	2008-12-31	11
-164	val_164	2008-12-31	11
-431	val_431	2008-12-31	11
-454	val_454	2008-12-31	11
-478	val_478	2008-12-31	11
-298	val_298	2008-12-31	11
-125	val_125	2008-12-31	11
-431	val_431	2008-12-31	11
-164	val_164	2008-12-31	11
-424	val_424	2008-12-31	11
-187	val_187	2008-12-31	11
-382	val_382	2008-12-31	11
-5	val_5	2008-12-31	11
-70	val_70	2008-12-31	11
-397	val_397	2008-12-31	11
-480	val_480	2008-12-31	11
-291	val_291	2008-12-31	11
-24	val_24	2008-12-31	11
-351	val_351	2008-12-31	11
-255	val_255	2008-12-31	11
-104	val_104	2008-12-31	11
-70	val_70	2008-12-31	11
-163	val_163	2008-12-31	11
-438	val_438	2008-12-31	11
-119	val_119	2008-12-31	11
-414	val_414	2008-12-31	11
-200	val_200	2008-12-31	11
-491	val_491	2008-12-31	11
-237	val_237	2008-12-31	11
-439	val_439	2008-12-31	11
-360	val_360	2008-12-31	11
-248	val_248	2008-12-31	11
-479	val_479	2008-12-31	11
-305	val_305	2008-12-31	11
-417	val_417	2008-12-31	11
-199	val_199	2008-12-31	11
-444	val_444	2008-12-31	11
-120	val_120	2008-12-31	11
-429	val_429	2008-12-31	11
-169	val_169	2008-12-31	11
-443	val_443	2008-12-31	11
-323	val_323	2008-12-31	11
-325	val_325	2008-12-31	11
-277	val_277	2008-12-31	11
-230	val_230	2008-12-31	11
-478	val_478	2008-12-31	11
-178	val_178	2008-12-31	11
-468	val_468	2008-12-31	11
-310	val_310	2008-12-31	11
-317	val_317	2008-12-31	11
-333	val_333	2008-12-31	11
-493	val_493	2008-12-31	11
-460	val_460	2008-12-31	11
-207	val_207	2008-12-31	11
-249	val_249	2008-12-31	11
-265	val_265	2008-12-31	11
-480	val_480	2008-12-31	11
-83	val_83	2008-12-31	11
-136	val_136	2008-12-31	11
-353	val_353	2008-12-31	11
-172	val_172	2008-12-31	11
-214	val_214	2008-12-31	11
-462	val_462	2008-12-31	11
-233	val_233	2008-12-31	11
-406	val_406	2008-12-31	11
-133	val_133	2008-12-31	11
-175	val_175	2008-12-31	11
-189	val_189	2008-12-31	11
-454	val_454	2008-12-31	11
-375	val_375	2008-12-31	11
-401	val_401	2008-12-31	11
-421	val_421	2008-12-31	11
-407	val_407	2008-12-31	11
-384	val_384	2008-12-31	11
-256	val_256	2008-12-31	11
-26	val_26	2008-12-31	11
-134	val_134	2008-12-31	11
-67	val_67	2008-12-31	11
-384	val_384	2008-12-31	11
-379	val_379	2008-12-31	11
-18	val_18	2008-12-31	11
-462	val_462	2008-12-31	11
-492	val_492	2008-12-31	11
-100	val_100	2008-12-31	11
-298	val_298	2008-12-31	11
-9	val_9	2008-12-31	11
-341	val_341	2008-12-31	11
-498	val_498	2008-12-31	11
-146	val_146	2008-12-31	11
-458	val_458	2008-12-31	11
-362	val_362	2008-12-31	11
-186	val_186	2008-12-31	11
-285	val_285	2008-12-31	11
-348	val_348	2008-12-31	11
-167	val_167	2008-12-31	11
-18	val_18	2008-12-31	11
-273	val_273	2008-12-31	11
-183	val_183	2008-12-31	11
-281	val_281	2008-12-31	11
-344	val_344	2008-12-31	11
-97	val_97	2008-12-31	11
-469	val_469	2008-12-31	11
-315	val_315	2008-12-31	11
-84	val_84	2008-12-31	11
-28	val_28	2008-12-31	11
-37	val_37	2008-12-31	11
-448	val_448	2008-12-31	11
-152	val_152	2008-12-31	11
-348	val_348	2008-12-31	11
-307	val_307	2008-12-31	11
-194	val_194	2008-12-31	11
-414	val_414	2008-12-31	11
-477	val_477	2008-12-31	11
-222	val_222	2008-12-31	11
-126	val_126	2008-12-31	11
-90	val_90	2008-12-31	11
-169	val_169	2008-12-31	11
-403	val_403	2008-12-31	11
-400	val_400	2008-12-31	11
-200	val_200	2008-12-31	11
-97	val_97	2008-12-31	11
-238	val_238	2008-12-31	12
-86	val_86	2008-12-31	12
-311	val_311	2008-12-31	12
-27	val_27	2008-12-31	12
-165	val_165	2008-12-31	12
-409	val_409	2008-12-31	12
-255	val_255	2008-12-31	12
-278	val_278	2008-12-31	12
-98	val_98	2008-12-31	12
-484	val_484	2008-12-31	12
-265	val_265	2008-12-31	12
-193	val_193	2008-12-31	12
-401	val_401	2008-12-31	12
-150	val_150	2008-12-31	12
-273	val_273	2008-12-31	12
-224	val_224	2008-12-31	12
-369	val_369	2008-12-31	12
-66	val_66	2008-12-31	12
-128	val_128	2008-12-31	12
-213	val_213	2008-12-31	12
-146	val_146	2008-12-31	12
-406	val_406	2008-12-31	12
-429	val_429	2008-12-31	12
-374	val_374	2008-12-31	12
-152	val_152	2008-12-31	12
-469	val_469	2008-12-31	12
-145	val_145	2008-12-31	12
-495	val_495	2008-12-31	12
-37	val_37	2008-12-31	12
-327	val_327	2008-12-31	12
-281	val_281	2008-12-31	12
-277	val_277	2008-12-31	12
-209	val_209	2008-12-31	12
-15	val_15	2008-12-31	12
-82	val_82	2008-12-31	12
-403	val_403	2008-12-31	12
-166	val_166	2008-12-31	12
-417	val_417	2008-12-31	12
-430	val_430	2008-12-31	12
-252	val_252	2008-12-31	12
-292	val_292	2008-12-31	12
-219	val_219	2008-12-31	12
-287	val_287	2008-12-31	12
-153	val_153	2008-12-31	12
-193	val_193	2008-12-31	12
-338	val_338	2008-12-31	12
-446	val_446	2008-12-31	12
-459	val_459	2008-12-31	12
-394	val_394	2008-12-31	12
-237	val_237	2008-12-31	12
-482	val_482	2008-12-31	12
-174	val_174	2008-12-31	12
-413	val_413	2008-12-31	12
-494	val_494	2008-12-31	12
-207	val_207	2008-12-31	12
-199	val_199	2008-12-31	12
-466	val_466	2008-12-31	12
-208	val_208	2008-12-31	12
-174	val_174	2008-12-31	12
-399	val_399	2008-12-31	12
-396	val_396	2008-12-31	12
-247	val_247	2008-12-31	12
-417	val_417	2008-12-31	12
-489	val_489	2008-12-31	12
-162	val_162	2008-12-31	12
-377	val_377	2008-12-31	12
-397	val_397	2008-12-31	12
-309	val_309	2008-12-31	12
-365	val_365	2008-12-31	12
-266	val_266	2008-12-31	12
-439	val_439	2008-12-31	12
-342	val_342	2008-12-31	12
-367	val_367	2008-12-31	12
-325	val_325	2008-12-31	12
-167	val_167	2008-12-31	12
-195	val_195	2008-12-31	12
-475	val_475	2008-12-31	12
-17	val_17	2008-12-31	12
-113	val_113	2008-12-31	12
-155	val_155	2008-12-31	12
-203	val_203	2008-12-31	12
-339	val_339	2008-12-31	12
-0	val_0	2008-12-31	12
-455	val_455	2008-12-31	12
-128	val_128	2008-12-31	12
-311	val_311	2008-12-31	12
-316	val_316	2008-12-31	12
-57	val_57	2008-12-31	12
-302	val_302	2008-12-31	12
-205	val_205	2008-12-31	12
-149	val_149	2008-12-31	12
-438	val_438	2008-12-31	12
-345	val_345	2008-12-31	12
-129	val_129	2008-12-31	12
-170	val_170	2008-12-31	12
-20	val_20	2008-12-31	12
-489	val_489	2008-12-31	12
-157	val_157	2008-12-31	12
-378	val_378	2008-12-31	12
-221	val_221	2008-12-31	12
-92	val_92	2008-12-31	12
-111	val_111	2008-12-31	12
-47	val_47	2008-12-31	12
-72	val_72	2008-12-31	12
-4	val_4	2008-12-31	12
-280	val_280	2008-12-31	12
-35	val_35	2008-12-31	12
-427	val_427	2008-12-31	12
-277	val_277	2008-12-31	12
-208	val_208	2008-12-31	12
-356	val_356	2008-12-31	12
-399	val_399	2008-12-31	12
-169	val_169	2008-12-31	12
-382	val_382	2008-12-31	12
-498	val_498	2008-12-31	12
-125	val_125	2008-12-31	12
-386	val_386	2008-12-31	12
-437	val_437	2008-12-31	12
-469	val_469	2008-12-31	12
-192	val_192	2008-12-31	12
-286	val_286	2008-12-31	12
-187	val_187	2008-12-31	12
-176	val_176	2008-12-31	12
-54	val_54	2008-12-31	12
-459	val_459	2008-12-31	12
-51	val_51	2008-12-31	12
-138	val_138	2008-12-31	12
-103	val_103	2008-12-31	12
-239	val_239	2008-12-31	12
-213	val_213	2008-12-31	12
-216	val_216	2008-12-31	12
-430	val_430	2008-12-31	12
-278	val_278	2008-12-31	12
-176	val_176	2008-12-31	12
-289	val_289	2008-12-31	12
-221	val_221	2008-12-31	12
-65	val_65	2008-12-31	12
-318	val_318	2008-12-31	12
-332	val_332	2008-12-31	12
-311	val_311	2008-12-31	12
-275	val_275	2008-12-31	12
-137	val_137	2008-12-31	12
-241	val_241	2008-12-31	12
-83	val_83	2008-12-31	12
-333	val_333	2008-12-31	12
-180	val_180	2008-12-31	12
-284	val_284	2008-12-31	12
-12	val_12	2008-12-31	12
-230	val_230	2008-12-31	12
-181	val_181	2008-12-31	12
-67	val_67	2008-12-31	12
-260	val_260	2008-12-31	12
-404	val_404	2008-12-31	12
-384	val_384	2008-12-31	12
-489	val_489	2008-12-31	12
-353	val_353	2008-12-31	12
-373	val_373	2008-12-31	12
-272	val_272	2008-12-31	12
-138	val_138	2008-12-31	12
-217	val_217	2008-12-31	12
-84	val_84	2008-12-31	12
-348	val_348	2008-12-31	12
-466	val_466	2008-12-31	12
-58	val_58	2008-12-31	12
-8	val_8	2008-12-31	12
-411	val_411	2008-12-31	12
-230	val_230	2008-12-31	12
-208	val_208	2008-12-31	12
-348	val_348	2008-12-31	12
-24	val_24	2008-12-31	12
-463	val_463	2008-12-31	12
-431	val_431	2008-12-31	12
-179	val_179	2008-12-31	12
-172	val_172	2008-12-31	12
-42	val_42	2008-12-31	12
-129	val_129	2008-12-31	12
-158	val_158	2008-12-31	12
-119	val_119	2008-12-31	12
-496	val_496	2008-12-31	12
-0	val_0	2008-12-31	12
-322	val_322	2008-12-31	12
-197	val_197	2008-12-31	12
-468	val_468	2008-12-31	12
-393	val_393	2008-12-31	12
-454	val_454	2008-12-31	12
-100	val_100	2008-12-31	12
-298	val_298	2008-12-31	12
-199	val_199	2008-12-31	12
-191	val_191	2008-12-31	12
-418	val_418	2008-12-31	12
-96	val_96	2008-12-31	12
-26	val_26	2008-12-31	12
-165	val_165	2008-12-31	12
-327	val_327	2008-12-31	12
-230	val_230	2008-12-31	12
-205	val_205	2008-12-31	12
-120	val_120	2008-12-31	12
-131	val_131	2008-12-31	12
-51	val_51	2008-12-31	12
-404	val_404	2008-12-31	12
-43	val_43	2008-12-31	12
-436	val_436	2008-12-31	12
-156	val_156	2008-12-31	12
-469	val_469	2008-12-31	12
-468	val_468	2008-12-31	12
-308	val_308	2008-12-31	12
-95	val_95	2008-12-31	12
-196	val_196	2008-12-31	12
-288	val_288	2008-12-31	12
-481	val_481	2008-12-31	12
-457	val_457	2008-12-31	12
-98	val_98	2008-12-31	12
-282	val_282	2008-12-31	12
-197	val_197	2008-12-31	12
-187	val_187	2008-12-31	12
-318	val_318	2008-12-31	12
-318	val_318	2008-12-31	12
-409	val_409	2008-12-31	12
-470	val_470	2008-12-31	12
-137	val_137	2008-12-31	12
-369	val_369	2008-12-31	12
-316	val_316	2008-12-31	12
-169	val_169	2008-12-31	12
-413	val_413	2008-12-31	12
-85	val_85	2008-12-31	12
-77	val_77	2008-12-31	12
-0	val_0	2008-12-31	12
-490	val_490	2008-12-31	12
-87	val_87	2008-12-31	12
-364	val_364	2008-12-31	12
-179	val_179	2008-12-31	12
-118	val_118	2008-12-31	12
-134	val_134	2008-12-31	12
-395	val_395	2008-12-31	12
-282	val_282	2008-12-31	12
-138	val_138	2008-12-31	12
-238	val_238	2008-12-31	12
-419	val_419	2008-12-31	12
-15	val_15	2008-12-31	12
-118	val_118	2008-12-31	12
-72	val_72	2008-12-31	12
-90	val_90	2008-12-31	12
-307	val_307	2008-12-31	12
-19	val_19	2008-12-31	12
-435	val_435	2008-12-31	12
-10	val_10	2008-12-31	12
-277	val_277	2008-12-31	12
-273	val_273	2008-12-31	12
-306	val_306	2008-12-31	12
-224	val_224	2008-12-31	12
-309	val_309	2008-12-31	12
-389	val_389	2008-12-31	12
-327	val_327	2008-12-31	12
-242	val_242	2008-12-31	12
-369	val_369	2008-12-31	12
-392	val_392	2008-12-31	12
-272	val_272	2008-12-31	12
-331	val_331	2008-12-31	12
-401	val_401	2008-12-31	12
-242	val_242	2008-12-31	12
-452	val_452	2008-12-31	12
-177	val_177	2008-12-31	12
-226	val_226	2008-12-31	12
-5	val_5	2008-12-31	12
-497	val_497	2008-12-31	12
-402	val_402	2008-12-31	12
-396	val_396	2008-12-31	12
-317	val_317	2008-12-31	12
-395	val_395	2008-12-31	12
-58	val_58	2008-12-31	12
-35	val_35	2008-12-31	12
-336	val_336	2008-12-31	12
-95	val_95	2008-12-31	12
-11	val_11	2008-12-31	12
-168	val_168	2008-12-31	12
-34	val_34	2008-12-31	12
-229	val_229	2008-12-31	12
-233	val_233	2008-12-31	12
-143	val_143	2008-12-31	12
-472	val_472	2008-12-31	12
-322	val_322	2008-12-31	12
-498	val_498	2008-12-31	12
-160	val_160	2008-12-31	12
-195	val_195	2008-12-31	12
-42	val_42	2008-12-31	12
-321	val_321	2008-12-31	12
-430	val_430	2008-12-31	12
-119	val_119	2008-12-31	12
-489	val_489	2008-12-31	12
-458	val_458	2008-12-31	12
-78	val_78	2008-12-31	12
-76	val_76	2008-12-31	12
-41	val_41	2008-12-31	12
-223	val_223	2008-12-31	12
-492	val_492	2008-12-31	12
-149	val_149	2008-12-31	12
-449	val_449	2008-12-31	12
-218	val_218	2008-12-31	12
-228	val_228	2008-12-31	12
-138	val_138	2008-12-31	12
-453	val_453	2008-12-31	12
-30	val_30	2008-12-31	12
-209	val_209	2008-12-31	12
-64	val_64	2008-12-31	12
-468	val_468	2008-12-31	12
-76	val_76	2008-12-31	12
-74	val_74	2008-12-31	12
-342	val_342	2008-12-31	12
-69	val_69	2008-12-31	12
-230	val_230	2008-12-31	12
-33	val_33	2008-12-31	12
-368	val_368	2008-12-31	12
-103	val_103	2008-12-31	12
-296	val_296	2008-12-31	12
-113	val_113	2008-12-31	12
-216	val_216	2008-12-31	12
-367	val_367	2008-12-31	12
-344	val_344	2008-12-31	12
-167	val_167	2008-12-31	12
-274	val_274	2008-12-31	12
-219	val_219	2008-12-31	12
-239	val_239	2008-12-31	12
-485	val_485	2008-12-31	12
-116	val_116	2008-12-31	12
-223	val_223	2008-12-31	12
-256	val_256	2008-12-31	12
-263	val_263	2008-12-31	12
-70	val_70	2008-12-31	12
-487	val_487	2008-12-31	12
-480	val_480	2008-12-31	12
-401	val_401	2008-12-31	12
-288	val_288	2008-12-31	12
-191	val_191	2008-12-31	12
-5	val_5	2008-12-31	12
-244	val_244	2008-12-31	12
-438	val_438	2008-12-31	12
-128	val_128	2008-12-31	12
-467	val_467	2008-12-31	12
-432	val_432	2008-12-31	12
-202	val_202	2008-12-31	12
-316	val_316	2008-12-31	12
-229	val_229	2008-12-31	12
-469	val_469	2008-12-31	12
-463	val_463	2008-12-31	12
-280	val_280	2008-12-31	12
-2	val_2	2008-12-31	12
-35	val_35	2008-12-31	12
-283	val_283	2008-12-31	12
-331	val_331	2008-12-31	12
-235	val_235	2008-12-31	12
-80	val_80	2008-12-31	12
-44	val_44	2008-12-31	12
-193	val_193	2008-12-31	12
-321	val_321	2008-12-31	12
-335	val_335	2008-12-31	12
-104	val_104	2008-12-31	12
-466	val_466	2008-12-31	12
-366	val_366	2008-12-31	12
-175	val_175	2008-12-31	12
-403	val_403	2008-12-31	12
-483	val_483	2008-12-31	12
-53	val_53	2008-12-31	12
-105	val_105	2008-12-31	12
-257	val_257	2008-12-31	12
-406	val_406	2008-12-31	12
-409	val_409	2008-12-31	12
-190	val_190	2008-12-31	12
-406	val_406	2008-12-31	12
-401	val_401	2008-12-31	12
-114	val_114	2008-12-31	12
-258	val_258	2008-12-31	12
-90	val_90	2008-12-31	12
-203	val_203	2008-12-31	12
-262	val_262	2008-12-31	12
-348	val_348	2008-12-31	12
-424	val_424	2008-12-31	12
-12	val_12	2008-12-31	12
-396	val_396	2008-12-31	12
-201	val_201	2008-12-31	12
-217	val_217	2008-12-31	12
-164	val_164	2008-12-31	12
-431	val_431	2008-12-31	12
-454	val_454	2008-12-31	12
-478	val_478	2008-12-31	12
-298	val_298	2008-12-31	12
-125	val_125	2008-12-31	12
-431	val_431	2008-12-31	12
-164	val_164	2008-12-31	12
-424	val_424	2008-12-31	12
-187	val_187	2008-12-31	12
-382	val_382	2008-12-31	12
-5	val_5	2008-12-31	12
-70	val_70	2008-12-31	12
-397	val_397	2008-12-31	12
-480	val_480	2008-12-31	12
-291	val_291	2008-12-31	12
-24	val_24	2008-12-31	12
-351	val_351	2008-12-31	12
-255	val_255	2008-12-31	12
-104	val_104	2008-12-31	12
-70	val_70	2008-12-31	12
-163	val_163	2008-12-31	12
-438	val_438	2008-12-31	12
-119	val_119	2008-12-31	12
-414	val_414	2008-12-31	12
-200	val_200	2008-12-31	12
-491	val_491	2008-12-31	12
-237	val_237	2008-12-31	12
-439	val_439	2008-12-31	12
-360	val_360	2008-12-31	12
-248	val_248	2008-12-31	12
-479	val_479	2008-12-31	12
-305	val_305	2008-12-31	12
-417	val_417	2008-12-31	12
-199	val_199	2008-12-31	12
-444	val_444	2008-12-31	12
-120	val_120	2008-12-31	12
-429	val_429	2008-12-31	12
-169	val_169	2008-12-31	12
-443	val_443	2008-12-31	12
-323	val_323	2008-12-31	12
-325	val_325	2008-12-31	12
-277	val_277	2008-12-31	12
-230	val_230	2008-12-31	12
-478	val_478	2008-12-31	12
-178	val_178	2008-12-31	12
-468	val_468	2008-12-31	12
-310	val_310	2008-12-31	12
-317	val_317	2008-12-31	12
-333	val_333	2008-12-31	12
-493	val_493	2008-12-31	12
-460	val_460	2008-12-31	12
-207	val_207	2008-12-31	12
-249	val_249	2008-12-31	12
-265	val_265	2008-12-31	12
-480	val_480	2008-12-31	12
-83	val_83	2008-12-31	12
-136	val_136	2008-12-31	12
-353	val_353	2008-12-31	12
-172	val_172	2008-12-31	12
-214	val_214	2008-12-31	12
-462	val_462	2008-12-31	12
-233	val_233	2008-12-31	12
-406	val_406	2008-12-31	12
-133	val_133	2008-12-31	12
-175	val_175	2008-12-31	12
-189	val_189	2008-12-31	12
-454	val_454	2008-12-31	12
-375	val_375	2008-12-31	12
-401	val_401	2008-12-31	12
-421	val_421	2008-12-31	12
-407	val_407	2008-12-31	12
-384	val_384	2008-12-31	12
-256	val_256	2008-12-31	12
-26	val_26	2008-12-31	12
-134	val_134	2008-12-31	12
-67	val_67	2008-12-31	12
-384	val_384	2008-12-31	12
-379	val_379	2008-12-31	12
-18	val_18	2008-12-31	12
-462	val_462	2008-12-31	12
-492	val_492	2008-12-31	12
-100	val_100	2008-12-31	12
-298	val_298	2008-12-31	12
-9	val_9	2008-12-31	12
-341	val_341	2008-12-31	12
-498	val_498	2008-12-31	12
-146	val_146	2008-12-31	12
-458	val_458	2008-12-31	12
-362	val_362	2008-12-31	12
-186	val_186	2008-12-31	12
-285	val_285	2008-12-31	12
-348	val_348	2008-12-31	12
-167	val_167	2008-12-31	12
-18	val_18	2008-12-31	12
-273	val_273	2008-12-31	12
-183	val_183	2008-12-31	12
-281	val_281	2008-12-31	12
-344	val_344	2008-12-31	12
-97	val_97	2008-12-31	12
-469	val_469	2008-12-31	12
-315	val_315	2008-12-31	12
-84	val_84	2008-12-31	12
-28	val_28	2008-12-31	12
-37	val_37	2008-12-31	12
-448	val_448	2008-12-31	12
-152	val_152	2008-12-31	12
-348	val_348	2008-12-31	12
-307	val_307	2008-12-31	12
-194	val_194	2008-12-31	12
-414	val_414	2008-12-31	12
-477	val_477	2008-12-31	12
-222	val_222	2008-12-31	12
-126	val_126	2008-12-31	12
-90	val_90	2008-12-31	12
-169	val_169	2008-12-31	12
-403	val_403	2008-12-31	12
-400	val_400	2008-12-31	12
-200	val_200	2008-12-31	12
-97	val_97	2008-12-31	12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 b/sql/hive/src/test/resources/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3
index e0dcec08697349537a2197fab4249e5a04917891..bf80c353fd1ea050131878c82b2382322634bcce 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3
+++ b/sql/hive/src/test/resources/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3
@@ -1,12 +1,12 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-hr                  	string              	None                
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part1, dbName:default, owner:marmbrus, createTime:1390899591, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{p3=v3, transient_lastDdlTime=1390899591}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:nzhang_part1, dbName:default, owner:marmbrus, createTime:1413887363, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887363}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302
index e9c723bbd136e561e0ff75d4b248e96ccbf81e73..8c43153cf66f9f265bbe39da3810288e57771358 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302
+++ b/sql/hive/src/test/resources/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302
@@ -1,4 +1,4 @@
 ds=2008-04-08/hr=11
 ds=2008-04-08/hr=12
 ds=2008-04-09/hr=11
-ds=2008-04-09/hr=12
\ No newline at end of file
+ds=2008-04-09/hr=12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e b/sql/hive/src/test/resources/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e
index 9d3a522ff81dcdb016d3f7210f1393d61a71b2fb..63350bdd8517fb349ab2e9591ef183d3176b0283 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e
+++ b/sql/hive/src/test/resources/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e
@@ -1,12 +1,12 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-hr                  	string              	None                
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part10, dbName:default, owner:marmbrus, createTime:1389738873, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4075462935071533647/nzhang_part10, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1389738873}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:nzhang_part10, dbName:default, owner:marmbrus, createTime:1413887405, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part10, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887405}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7
+++ b/sql/hive/src/test/resources/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe
+++ b/sql/hive/src/test/resources/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967 b/sql/hive/src/test/resources/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967
index 574727266b1839c4475d7bc3edf944bd1ca62afe..1634adfc4d70d083df5abc5abbfd0853b23062c0 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967
+++ b/sql/hive/src/test/resources/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967
@@ -1,2 +1,2 @@
 ds=2008-12-31/hr=11
-ds=2008-12-31/hr=12
\ No newline at end of file
+ds=2008-12-31/hr=12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part10-9-a646fd106fe73b8753fe3fee86d56ebf b/sql/hive/src/test/resources/golden/load_dyn_part10-9-a646fd106fe73b8753fe3fee86d56ebf
new file mode 100644
index 0000000000000000000000000000000000000000..ee4601f59e708c4de203abc48fe418b32a3c8797
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/load_dyn_part10-9-a646fd106fe73b8753fe3fee86d56ebf
@@ -0,0 +1,1000 @@
+0	val_0	2008-12-31	11
+0	val_0	2008-12-31	11
+0	val_0	2008-12-31	11
+2	val_2	2008-12-31	11
+4	val_4	2008-12-31	11
+5	val_5	2008-12-31	11
+5	val_5	2008-12-31	11
+5	val_5	2008-12-31	11
+8	val_8	2008-12-31	11
+9	val_9	2008-12-31	11
+10	val_10	2008-12-31	11
+11	val_11	2008-12-31	11
+12	val_12	2008-12-31	11
+12	val_12	2008-12-31	11
+15	val_15	2008-12-31	11
+15	val_15	2008-12-31	11
+17	val_17	2008-12-31	11
+18	val_18	2008-12-31	11
+18	val_18	2008-12-31	11
+19	val_19	2008-12-31	11
+20	val_20	2008-12-31	11
+24	val_24	2008-12-31	11
+24	val_24	2008-12-31	11
+26	val_26	2008-12-31	11
+26	val_26	2008-12-31	11
+27	val_27	2008-12-31	11
+28	val_28	2008-12-31	11
+30	val_30	2008-12-31	11
+33	val_33	2008-12-31	11
+34	val_34	2008-12-31	11
+35	val_35	2008-12-31	11
+35	val_35	2008-12-31	11
+35	val_35	2008-12-31	11
+37	val_37	2008-12-31	11
+37	val_37	2008-12-31	11
+41	val_41	2008-12-31	11
+42	val_42	2008-12-31	11
+42	val_42	2008-12-31	11
+43	val_43	2008-12-31	11
+44	val_44	2008-12-31	11
+47	val_47	2008-12-31	11
+51	val_51	2008-12-31	11
+51	val_51	2008-12-31	11
+53	val_53	2008-12-31	11
+54	val_54	2008-12-31	11
+57	val_57	2008-12-31	11
+58	val_58	2008-12-31	11
+58	val_58	2008-12-31	11
+64	val_64	2008-12-31	11
+65	val_65	2008-12-31	11
+66	val_66	2008-12-31	11
+67	val_67	2008-12-31	11
+67	val_67	2008-12-31	11
+69	val_69	2008-12-31	11
+70	val_70	2008-12-31	11
+70	val_70	2008-12-31	11
+70	val_70	2008-12-31	11
+72	val_72	2008-12-31	11
+72	val_72	2008-12-31	11
+74	val_74	2008-12-31	11
+76	val_76	2008-12-31	11
+76	val_76	2008-12-31	11
+77	val_77	2008-12-31	11
+78	val_78	2008-12-31	11
+80	val_80	2008-12-31	11
+82	val_82	2008-12-31	11
+83	val_83	2008-12-31	11
+83	val_83	2008-12-31	11
+84	val_84	2008-12-31	11
+84	val_84	2008-12-31	11
+85	val_85	2008-12-31	11
+86	val_86	2008-12-31	11
+87	val_87	2008-12-31	11
+90	val_90	2008-12-31	11
+90	val_90	2008-12-31	11
+90	val_90	2008-12-31	11
+92	val_92	2008-12-31	11
+95	val_95	2008-12-31	11
+95	val_95	2008-12-31	11
+96	val_96	2008-12-31	11
+97	val_97	2008-12-31	11
+97	val_97	2008-12-31	11
+98	val_98	2008-12-31	11
+98	val_98	2008-12-31	11
+100	val_100	2008-12-31	11
+100	val_100	2008-12-31	11
+103	val_103	2008-12-31	11
+103	val_103	2008-12-31	11
+104	val_104	2008-12-31	11
+104	val_104	2008-12-31	11
+105	val_105	2008-12-31	11
+111	val_111	2008-12-31	11
+113	val_113	2008-12-31	11
+113	val_113	2008-12-31	11
+114	val_114	2008-12-31	11
+116	val_116	2008-12-31	11
+118	val_118	2008-12-31	11
+118	val_118	2008-12-31	11
+119	val_119	2008-12-31	11
+119	val_119	2008-12-31	11
+119	val_119	2008-12-31	11
+120	val_120	2008-12-31	11
+120	val_120	2008-12-31	11
+125	val_125	2008-12-31	11
+125	val_125	2008-12-31	11
+126	val_126	2008-12-31	11
+128	val_128	2008-12-31	11
+128	val_128	2008-12-31	11
+128	val_128	2008-12-31	11
+129	val_129	2008-12-31	11
+129	val_129	2008-12-31	11
+131	val_131	2008-12-31	11
+133	val_133	2008-12-31	11
+134	val_134	2008-12-31	11
+134	val_134	2008-12-31	11
+136	val_136	2008-12-31	11
+137	val_137	2008-12-31	11
+137	val_137	2008-12-31	11
+138	val_138	2008-12-31	11
+138	val_138	2008-12-31	11
+138	val_138	2008-12-31	11
+138	val_138	2008-12-31	11
+143	val_143	2008-12-31	11
+145	val_145	2008-12-31	11
+146	val_146	2008-12-31	11
+146	val_146	2008-12-31	11
+149	val_149	2008-12-31	11
+149	val_149	2008-12-31	11
+150	val_150	2008-12-31	11
+152	val_152	2008-12-31	11
+152	val_152	2008-12-31	11
+153	val_153	2008-12-31	11
+155	val_155	2008-12-31	11
+156	val_156	2008-12-31	11
+157	val_157	2008-12-31	11
+158	val_158	2008-12-31	11
+160	val_160	2008-12-31	11
+162	val_162	2008-12-31	11
+163	val_163	2008-12-31	11
+164	val_164	2008-12-31	11
+164	val_164	2008-12-31	11
+165	val_165	2008-12-31	11
+165	val_165	2008-12-31	11
+166	val_166	2008-12-31	11
+167	val_167	2008-12-31	11
+167	val_167	2008-12-31	11
+167	val_167	2008-12-31	11
+168	val_168	2008-12-31	11
+169	val_169	2008-12-31	11
+169	val_169	2008-12-31	11
+169	val_169	2008-12-31	11
+169	val_169	2008-12-31	11
+170	val_170	2008-12-31	11
+172	val_172	2008-12-31	11
+172	val_172	2008-12-31	11
+174	val_174	2008-12-31	11
+174	val_174	2008-12-31	11
+175	val_175	2008-12-31	11
+175	val_175	2008-12-31	11
+176	val_176	2008-12-31	11
+176	val_176	2008-12-31	11
+177	val_177	2008-12-31	11
+178	val_178	2008-12-31	11
+179	val_179	2008-12-31	11
+179	val_179	2008-12-31	11
+180	val_180	2008-12-31	11
+181	val_181	2008-12-31	11
+183	val_183	2008-12-31	11
+186	val_186	2008-12-31	11
+187	val_187	2008-12-31	11
+187	val_187	2008-12-31	11
+187	val_187	2008-12-31	11
+189	val_189	2008-12-31	11
+190	val_190	2008-12-31	11
+191	val_191	2008-12-31	11
+191	val_191	2008-12-31	11
+192	val_192	2008-12-31	11
+193	val_193	2008-12-31	11
+193	val_193	2008-12-31	11
+193	val_193	2008-12-31	11
+194	val_194	2008-12-31	11
+195	val_195	2008-12-31	11
+195	val_195	2008-12-31	11
+196	val_196	2008-12-31	11
+197	val_197	2008-12-31	11
+197	val_197	2008-12-31	11
+199	val_199	2008-12-31	11
+199	val_199	2008-12-31	11
+199	val_199	2008-12-31	11
+200	val_200	2008-12-31	11
+200	val_200	2008-12-31	11
+201	val_201	2008-12-31	11
+202	val_202	2008-12-31	11
+203	val_203	2008-12-31	11
+203	val_203	2008-12-31	11
+205	val_205	2008-12-31	11
+205	val_205	2008-12-31	11
+207	val_207	2008-12-31	11
+207	val_207	2008-12-31	11
+208	val_208	2008-12-31	11
+208	val_208	2008-12-31	11
+208	val_208	2008-12-31	11
+209	val_209	2008-12-31	11
+209	val_209	2008-12-31	11
+213	val_213	2008-12-31	11
+213	val_213	2008-12-31	11
+214	val_214	2008-12-31	11
+216	val_216	2008-12-31	11
+216	val_216	2008-12-31	11
+217	val_217	2008-12-31	11
+217	val_217	2008-12-31	11
+218	val_218	2008-12-31	11
+219	val_219	2008-12-31	11
+219	val_219	2008-12-31	11
+221	val_221	2008-12-31	11
+221	val_221	2008-12-31	11
+222	val_222	2008-12-31	11
+223	val_223	2008-12-31	11
+223	val_223	2008-12-31	11
+224	val_224	2008-12-31	11
+224	val_224	2008-12-31	11
+226	val_226	2008-12-31	11
+228	val_228	2008-12-31	11
+229	val_229	2008-12-31	11
+229	val_229	2008-12-31	11
+230	val_230	2008-12-31	11
+230	val_230	2008-12-31	11
+230	val_230	2008-12-31	11
+230	val_230	2008-12-31	11
+230	val_230	2008-12-31	11
+233	val_233	2008-12-31	11
+233	val_233	2008-12-31	11
+235	val_235	2008-12-31	11
+237	val_237	2008-12-31	11
+237	val_237	2008-12-31	11
+238	val_238	2008-12-31	11
+238	val_238	2008-12-31	11
+239	val_239	2008-12-31	11
+239	val_239	2008-12-31	11
+241	val_241	2008-12-31	11
+242	val_242	2008-12-31	11
+242	val_242	2008-12-31	11
+244	val_244	2008-12-31	11
+247	val_247	2008-12-31	11
+248	val_248	2008-12-31	11
+249	val_249	2008-12-31	11
+252	val_252	2008-12-31	11
+255	val_255	2008-12-31	11
+255	val_255	2008-12-31	11
+256	val_256	2008-12-31	11
+256	val_256	2008-12-31	11
+257	val_257	2008-12-31	11
+258	val_258	2008-12-31	11
+260	val_260	2008-12-31	11
+262	val_262	2008-12-31	11
+263	val_263	2008-12-31	11
+265	val_265	2008-12-31	11
+265	val_265	2008-12-31	11
+266	val_266	2008-12-31	11
+272	val_272	2008-12-31	11
+272	val_272	2008-12-31	11
+273	val_273	2008-12-31	11
+273	val_273	2008-12-31	11
+273	val_273	2008-12-31	11
+274	val_274	2008-12-31	11
+275	val_275	2008-12-31	11
+277	val_277	2008-12-31	11
+277	val_277	2008-12-31	11
+277	val_277	2008-12-31	11
+277	val_277	2008-12-31	11
+278	val_278	2008-12-31	11
+278	val_278	2008-12-31	11
+280	val_280	2008-12-31	11
+280	val_280	2008-12-31	11
+281	val_281	2008-12-31	11
+281	val_281	2008-12-31	11
+282	val_282	2008-12-31	11
+282	val_282	2008-12-31	11
+283	val_283	2008-12-31	11
+284	val_284	2008-12-31	11
+285	val_285	2008-12-31	11
+286	val_286	2008-12-31	11
+287	val_287	2008-12-31	11
+288	val_288	2008-12-31	11
+288	val_288	2008-12-31	11
+289	val_289	2008-12-31	11
+291	val_291	2008-12-31	11
+292	val_292	2008-12-31	11
+296	val_296	2008-12-31	11
+298	val_298	2008-12-31	11
+298	val_298	2008-12-31	11
+298	val_298	2008-12-31	11
+302	val_302	2008-12-31	11
+305	val_305	2008-12-31	11
+306	val_306	2008-12-31	11
+307	val_307	2008-12-31	11
+307	val_307	2008-12-31	11
+308	val_308	2008-12-31	11
+309	val_309	2008-12-31	11
+309	val_309	2008-12-31	11
+310	val_310	2008-12-31	11
+311	val_311	2008-12-31	11
+311	val_311	2008-12-31	11
+311	val_311	2008-12-31	11
+315	val_315	2008-12-31	11
+316	val_316	2008-12-31	11
+316	val_316	2008-12-31	11
+316	val_316	2008-12-31	11
+317	val_317	2008-12-31	11
+317	val_317	2008-12-31	11
+318	val_318	2008-12-31	11
+318	val_318	2008-12-31	11
+318	val_318	2008-12-31	11
+321	val_321	2008-12-31	11
+321	val_321	2008-12-31	11
+322	val_322	2008-12-31	11
+322	val_322	2008-12-31	11
+323	val_323	2008-12-31	11
+325	val_325	2008-12-31	11
+325	val_325	2008-12-31	11
+327	val_327	2008-12-31	11
+327	val_327	2008-12-31	11
+327	val_327	2008-12-31	11
+331	val_331	2008-12-31	11
+331	val_331	2008-12-31	11
+332	val_332	2008-12-31	11
+333	val_333	2008-12-31	11
+333	val_333	2008-12-31	11
+335	val_335	2008-12-31	11
+336	val_336	2008-12-31	11
+338	val_338	2008-12-31	11
+339	val_339	2008-12-31	11
+341	val_341	2008-12-31	11
+342	val_342	2008-12-31	11
+342	val_342	2008-12-31	11
+344	val_344	2008-12-31	11
+344	val_344	2008-12-31	11
+345	val_345	2008-12-31	11
+348	val_348	2008-12-31	11
+348	val_348	2008-12-31	11
+348	val_348	2008-12-31	11
+348	val_348	2008-12-31	11
+348	val_348	2008-12-31	11
+351	val_351	2008-12-31	11
+353	val_353	2008-12-31	11
+353	val_353	2008-12-31	11
+356	val_356	2008-12-31	11
+360	val_360	2008-12-31	11
+362	val_362	2008-12-31	11
+364	val_364	2008-12-31	11
+365	val_365	2008-12-31	11
+366	val_366	2008-12-31	11
+367	val_367	2008-12-31	11
+367	val_367	2008-12-31	11
+368	val_368	2008-12-31	11
+369	val_369	2008-12-31	11
+369	val_369	2008-12-31	11
+369	val_369	2008-12-31	11
+373	val_373	2008-12-31	11
+374	val_374	2008-12-31	11
+375	val_375	2008-12-31	11
+377	val_377	2008-12-31	11
+378	val_378	2008-12-31	11
+379	val_379	2008-12-31	11
+382	val_382	2008-12-31	11
+382	val_382	2008-12-31	11
+384	val_384	2008-12-31	11
+384	val_384	2008-12-31	11
+384	val_384	2008-12-31	11
+386	val_386	2008-12-31	11
+389	val_389	2008-12-31	11
+392	val_392	2008-12-31	11
+393	val_393	2008-12-31	11
+394	val_394	2008-12-31	11
+395	val_395	2008-12-31	11
+395	val_395	2008-12-31	11
+396	val_396	2008-12-31	11
+396	val_396	2008-12-31	11
+396	val_396	2008-12-31	11
+397	val_397	2008-12-31	11
+397	val_397	2008-12-31	11
+399	val_399	2008-12-31	11
+399	val_399	2008-12-31	11
+400	val_400	2008-12-31	11
+401	val_401	2008-12-31	11
+401	val_401	2008-12-31	11
+401	val_401	2008-12-31	11
+401	val_401	2008-12-31	11
+401	val_401	2008-12-31	11
+402	val_402	2008-12-31	11
+403	val_403	2008-12-31	11
+403	val_403	2008-12-31	11
+403	val_403	2008-12-31	11
+404	val_404	2008-12-31	11
+404	val_404	2008-12-31	11
+406	val_406	2008-12-31	11
+406	val_406	2008-12-31	11
+406	val_406	2008-12-31	11
+406	val_406	2008-12-31	11
+407	val_407	2008-12-31	11
+409	val_409	2008-12-31	11
+409	val_409	2008-12-31	11
+409	val_409	2008-12-31	11
+411	val_411	2008-12-31	11
+413	val_413	2008-12-31	11
+413	val_413	2008-12-31	11
+414	val_414	2008-12-31	11
+414	val_414	2008-12-31	11
+417	val_417	2008-12-31	11
+417	val_417	2008-12-31	11
+417	val_417	2008-12-31	11
+418	val_418	2008-12-31	11
+419	val_419	2008-12-31	11
+421	val_421	2008-12-31	11
+424	val_424	2008-12-31	11
+424	val_424	2008-12-31	11
+427	val_427	2008-12-31	11
+429	val_429	2008-12-31	11
+429	val_429	2008-12-31	11
+430	val_430	2008-12-31	11
+430	val_430	2008-12-31	11
+430	val_430	2008-12-31	11
+431	val_431	2008-12-31	11
+431	val_431	2008-12-31	11
+431	val_431	2008-12-31	11
+432	val_432	2008-12-31	11
+435	val_435	2008-12-31	11
+436	val_436	2008-12-31	11
+437	val_437	2008-12-31	11
+438	val_438	2008-12-31	11
+438	val_438	2008-12-31	11
+438	val_438	2008-12-31	11
+439	val_439	2008-12-31	11
+439	val_439	2008-12-31	11
+443	val_443	2008-12-31	11
+444	val_444	2008-12-31	11
+446	val_446	2008-12-31	11
+448	val_448	2008-12-31	11
+449	val_449	2008-12-31	11
+452	val_452	2008-12-31	11
+453	val_453	2008-12-31	11
+454	val_454	2008-12-31	11
+454	val_454	2008-12-31	11
+454	val_454	2008-12-31	11
+455	val_455	2008-12-31	11
+457	val_457	2008-12-31	11
+458	val_458	2008-12-31	11
+458	val_458	2008-12-31	11
+459	val_459	2008-12-31	11
+459	val_459	2008-12-31	11
+460	val_460	2008-12-31	11
+462	val_462	2008-12-31	11
+462	val_462	2008-12-31	11
+463	val_463	2008-12-31	11
+463	val_463	2008-12-31	11
+466	val_466	2008-12-31	11
+466	val_466	2008-12-31	11
+466	val_466	2008-12-31	11
+467	val_467	2008-12-31	11
+468	val_468	2008-12-31	11
+468	val_468	2008-12-31	11
+468	val_468	2008-12-31	11
+468	val_468	2008-12-31	11
+469	val_469	2008-12-31	11
+469	val_469	2008-12-31	11
+469	val_469	2008-12-31	11
+469	val_469	2008-12-31	11
+469	val_469	2008-12-31	11
+470	val_470	2008-12-31	11
+472	val_472	2008-12-31	11
+475	val_475	2008-12-31	11
+477	val_477	2008-12-31	11
+478	val_478	2008-12-31	11
+478	val_478	2008-12-31	11
+479	val_479	2008-12-31	11
+480	val_480	2008-12-31	11
+480	val_480	2008-12-31	11
+480	val_480	2008-12-31	11
+481	val_481	2008-12-31	11
+482	val_482	2008-12-31	11
+483	val_483	2008-12-31	11
+484	val_484	2008-12-31	11
+485	val_485	2008-12-31	11
+487	val_487	2008-12-31	11
+489	val_489	2008-12-31	11
+489	val_489	2008-12-31	11
+489	val_489	2008-12-31	11
+489	val_489	2008-12-31	11
+490	val_490	2008-12-31	11
+491	val_491	2008-12-31	11
+492	val_492	2008-12-31	11
+492	val_492	2008-12-31	11
+493	val_493	2008-12-31	11
+494	val_494	2008-12-31	11
+495	val_495	2008-12-31	11
+496	val_496	2008-12-31	11
+497	val_497	2008-12-31	11
+498	val_498	2008-12-31	11
+498	val_498	2008-12-31	11
+498	val_498	2008-12-31	11
+0	val_0	2008-12-31	12
+0	val_0	2008-12-31	12
+0	val_0	2008-12-31	12
+2	val_2	2008-12-31	12
+4	val_4	2008-12-31	12
+5	val_5	2008-12-31	12
+5	val_5	2008-12-31	12
+5	val_5	2008-12-31	12
+8	val_8	2008-12-31	12
+9	val_9	2008-12-31	12
+10	val_10	2008-12-31	12
+11	val_11	2008-12-31	12
+12	val_12	2008-12-31	12
+12	val_12	2008-12-31	12
+15	val_15	2008-12-31	12
+15	val_15	2008-12-31	12
+17	val_17	2008-12-31	12
+18	val_18	2008-12-31	12
+18	val_18	2008-12-31	12
+19	val_19	2008-12-31	12
+20	val_20	2008-12-31	12
+24	val_24	2008-12-31	12
+24	val_24	2008-12-31	12
+26	val_26	2008-12-31	12
+26	val_26	2008-12-31	12
+27	val_27	2008-12-31	12
+28	val_28	2008-12-31	12
+30	val_30	2008-12-31	12
+33	val_33	2008-12-31	12
+34	val_34	2008-12-31	12
+35	val_35	2008-12-31	12
+35	val_35	2008-12-31	12
+35	val_35	2008-12-31	12
+37	val_37	2008-12-31	12
+37	val_37	2008-12-31	12
+41	val_41	2008-12-31	12
+42	val_42	2008-12-31	12
+42	val_42	2008-12-31	12
+43	val_43	2008-12-31	12
+44	val_44	2008-12-31	12
+47	val_47	2008-12-31	12
+51	val_51	2008-12-31	12
+51	val_51	2008-12-31	12
+53	val_53	2008-12-31	12
+54	val_54	2008-12-31	12
+57	val_57	2008-12-31	12
+58	val_58	2008-12-31	12
+58	val_58	2008-12-31	12
+64	val_64	2008-12-31	12
+65	val_65	2008-12-31	12
+66	val_66	2008-12-31	12
+67	val_67	2008-12-31	12
+67	val_67	2008-12-31	12
+69	val_69	2008-12-31	12
+70	val_70	2008-12-31	12
+70	val_70	2008-12-31	12
+70	val_70	2008-12-31	12
+72	val_72	2008-12-31	12
+72	val_72	2008-12-31	12
+74	val_74	2008-12-31	12
+76	val_76	2008-12-31	12
+76	val_76	2008-12-31	12
+77	val_77	2008-12-31	12
+78	val_78	2008-12-31	12
+80	val_80	2008-12-31	12
+82	val_82	2008-12-31	12
+83	val_83	2008-12-31	12
+83	val_83	2008-12-31	12
+84	val_84	2008-12-31	12
+84	val_84	2008-12-31	12
+85	val_85	2008-12-31	12
+86	val_86	2008-12-31	12
+87	val_87	2008-12-31	12
+90	val_90	2008-12-31	12
+90	val_90	2008-12-31	12
+90	val_90	2008-12-31	12
+92	val_92	2008-12-31	12
+95	val_95	2008-12-31	12
+95	val_95	2008-12-31	12
+96	val_96	2008-12-31	12
+97	val_97	2008-12-31	12
+97	val_97	2008-12-31	12
+98	val_98	2008-12-31	12
+98	val_98	2008-12-31	12
+100	val_100	2008-12-31	12
+100	val_100	2008-12-31	12
+103	val_103	2008-12-31	12
+103	val_103	2008-12-31	12
+104	val_104	2008-12-31	12
+104	val_104	2008-12-31	12
+105	val_105	2008-12-31	12
+111	val_111	2008-12-31	12
+113	val_113	2008-12-31	12
+113	val_113	2008-12-31	12
+114	val_114	2008-12-31	12
+116	val_116	2008-12-31	12
+118	val_118	2008-12-31	12
+118	val_118	2008-12-31	12
+119	val_119	2008-12-31	12
+119	val_119	2008-12-31	12
+119	val_119	2008-12-31	12
+120	val_120	2008-12-31	12
+120	val_120	2008-12-31	12
+125	val_125	2008-12-31	12
+125	val_125	2008-12-31	12
+126	val_126	2008-12-31	12
+128	val_128	2008-12-31	12
+128	val_128	2008-12-31	12
+128	val_128	2008-12-31	12
+129	val_129	2008-12-31	12
+129	val_129	2008-12-31	12
+131	val_131	2008-12-31	12
+133	val_133	2008-12-31	12
+134	val_134	2008-12-31	12
+134	val_134	2008-12-31	12
+136	val_136	2008-12-31	12
+137	val_137	2008-12-31	12
+137	val_137	2008-12-31	12
+138	val_138	2008-12-31	12
+138	val_138	2008-12-31	12
+138	val_138	2008-12-31	12
+138	val_138	2008-12-31	12
+143	val_143	2008-12-31	12
+145	val_145	2008-12-31	12
+146	val_146	2008-12-31	12
+146	val_146	2008-12-31	12
+149	val_149	2008-12-31	12
+149	val_149	2008-12-31	12
+150	val_150	2008-12-31	12
+152	val_152	2008-12-31	12
+152	val_152	2008-12-31	12
+153	val_153	2008-12-31	12
+155	val_155	2008-12-31	12
+156	val_156	2008-12-31	12
+157	val_157	2008-12-31	12
+158	val_158	2008-12-31	12
+160	val_160	2008-12-31	12
+162	val_162	2008-12-31	12
+163	val_163	2008-12-31	12
+164	val_164	2008-12-31	12
+164	val_164	2008-12-31	12
+165	val_165	2008-12-31	12
+165	val_165	2008-12-31	12
+166	val_166	2008-12-31	12
+167	val_167	2008-12-31	12
+167	val_167	2008-12-31	12
+167	val_167	2008-12-31	12
+168	val_168	2008-12-31	12
+169	val_169	2008-12-31	12
+169	val_169	2008-12-31	12
+169	val_169	2008-12-31	12
+169	val_169	2008-12-31	12
+170	val_170	2008-12-31	12
+172	val_172	2008-12-31	12
+172	val_172	2008-12-31	12
+174	val_174	2008-12-31	12
+174	val_174	2008-12-31	12
+175	val_175	2008-12-31	12
+175	val_175	2008-12-31	12
+176	val_176	2008-12-31	12
+176	val_176	2008-12-31	12
+177	val_177	2008-12-31	12
+178	val_178	2008-12-31	12
+179	val_179	2008-12-31	12
+179	val_179	2008-12-31	12
+180	val_180	2008-12-31	12
+181	val_181	2008-12-31	12
+183	val_183	2008-12-31	12
+186	val_186	2008-12-31	12
+187	val_187	2008-12-31	12
+187	val_187	2008-12-31	12
+187	val_187	2008-12-31	12
+189	val_189	2008-12-31	12
+190	val_190	2008-12-31	12
+191	val_191	2008-12-31	12
+191	val_191	2008-12-31	12
+192	val_192	2008-12-31	12
+193	val_193	2008-12-31	12
+193	val_193	2008-12-31	12
+193	val_193	2008-12-31	12
+194	val_194	2008-12-31	12
+195	val_195	2008-12-31	12
+195	val_195	2008-12-31	12
+196	val_196	2008-12-31	12
+197	val_197	2008-12-31	12
+197	val_197	2008-12-31	12
+199	val_199	2008-12-31	12
+199	val_199	2008-12-31	12
+199	val_199	2008-12-31	12
+200	val_200	2008-12-31	12
+200	val_200	2008-12-31	12
+201	val_201	2008-12-31	12
+202	val_202	2008-12-31	12
+203	val_203	2008-12-31	12
+203	val_203	2008-12-31	12
+205	val_205	2008-12-31	12
+205	val_205	2008-12-31	12
+207	val_207	2008-12-31	12
+207	val_207	2008-12-31	12
+208	val_208	2008-12-31	12
+208	val_208	2008-12-31	12
+208	val_208	2008-12-31	12
+209	val_209	2008-12-31	12
+209	val_209	2008-12-31	12
+213	val_213	2008-12-31	12
+213	val_213	2008-12-31	12
+214	val_214	2008-12-31	12
+216	val_216	2008-12-31	12
+216	val_216	2008-12-31	12
+217	val_217	2008-12-31	12
+217	val_217	2008-12-31	12
+218	val_218	2008-12-31	12
+219	val_219	2008-12-31	12
+219	val_219	2008-12-31	12
+221	val_221	2008-12-31	12
+221	val_221	2008-12-31	12
+222	val_222	2008-12-31	12
+223	val_223	2008-12-31	12
+223	val_223	2008-12-31	12
+224	val_224	2008-12-31	12
+224	val_224	2008-12-31	12
+226	val_226	2008-12-31	12
+228	val_228	2008-12-31	12
+229	val_229	2008-12-31	12
+229	val_229	2008-12-31	12
+230	val_230	2008-12-31	12
+230	val_230	2008-12-31	12
+230	val_230	2008-12-31	12
+230	val_230	2008-12-31	12
+230	val_230	2008-12-31	12
+233	val_233	2008-12-31	12
+233	val_233	2008-12-31	12
+235	val_235	2008-12-31	12
+237	val_237	2008-12-31	12
+237	val_237	2008-12-31	12
+238	val_238	2008-12-31	12
+238	val_238	2008-12-31	12
+239	val_239	2008-12-31	12
+239	val_239	2008-12-31	12
+241	val_241	2008-12-31	12
+242	val_242	2008-12-31	12
+242	val_242	2008-12-31	12
+244	val_244	2008-12-31	12
+247	val_247	2008-12-31	12
+248	val_248	2008-12-31	12
+249	val_249	2008-12-31	12
+252	val_252	2008-12-31	12
+255	val_255	2008-12-31	12
+255	val_255	2008-12-31	12
+256	val_256	2008-12-31	12
+256	val_256	2008-12-31	12
+257	val_257	2008-12-31	12
+258	val_258	2008-12-31	12
+260	val_260	2008-12-31	12
+262	val_262	2008-12-31	12
+263	val_263	2008-12-31	12
+265	val_265	2008-12-31	12
+265	val_265	2008-12-31	12
+266	val_266	2008-12-31	12
+272	val_272	2008-12-31	12
+272	val_272	2008-12-31	12
+273	val_273	2008-12-31	12
+273	val_273	2008-12-31	12
+273	val_273	2008-12-31	12
+274	val_274	2008-12-31	12
+275	val_275	2008-12-31	12
+277	val_277	2008-12-31	12
+277	val_277	2008-12-31	12
+277	val_277	2008-12-31	12
+277	val_277	2008-12-31	12
+278	val_278	2008-12-31	12
+278	val_278	2008-12-31	12
+280	val_280	2008-12-31	12
+280	val_280	2008-12-31	12
+281	val_281	2008-12-31	12
+281	val_281	2008-12-31	12
+282	val_282	2008-12-31	12
+282	val_282	2008-12-31	12
+283	val_283	2008-12-31	12
+284	val_284	2008-12-31	12
+285	val_285	2008-12-31	12
+286	val_286	2008-12-31	12
+287	val_287	2008-12-31	12
+288	val_288	2008-12-31	12
+288	val_288	2008-12-31	12
+289	val_289	2008-12-31	12
+291	val_291	2008-12-31	12
+292	val_292	2008-12-31	12
+296	val_296	2008-12-31	12
+298	val_298	2008-12-31	12
+298	val_298	2008-12-31	12
+298	val_298	2008-12-31	12
+302	val_302	2008-12-31	12
+305	val_305	2008-12-31	12
+306	val_306	2008-12-31	12
+307	val_307	2008-12-31	12
+307	val_307	2008-12-31	12
+308	val_308	2008-12-31	12
+309	val_309	2008-12-31	12
+309	val_309	2008-12-31	12
+310	val_310	2008-12-31	12
+311	val_311	2008-12-31	12
+311	val_311	2008-12-31	12
+311	val_311	2008-12-31	12
+315	val_315	2008-12-31	12
+316	val_316	2008-12-31	12
+316	val_316	2008-12-31	12
+316	val_316	2008-12-31	12
+317	val_317	2008-12-31	12
+317	val_317	2008-12-31	12
+318	val_318	2008-12-31	12
+318	val_318	2008-12-31	12
+318	val_318	2008-12-31	12
+321	val_321	2008-12-31	12
+321	val_321	2008-12-31	12
+322	val_322	2008-12-31	12
+322	val_322	2008-12-31	12
+323	val_323	2008-12-31	12
+325	val_325	2008-12-31	12
+325	val_325	2008-12-31	12
+327	val_327	2008-12-31	12
+327	val_327	2008-12-31	12
+327	val_327	2008-12-31	12
+331	val_331	2008-12-31	12
+331	val_331	2008-12-31	12
+332	val_332	2008-12-31	12
+333	val_333	2008-12-31	12
+333	val_333	2008-12-31	12
+335	val_335	2008-12-31	12
+336	val_336	2008-12-31	12
+338	val_338	2008-12-31	12
+339	val_339	2008-12-31	12
+341	val_341	2008-12-31	12
+342	val_342	2008-12-31	12
+342	val_342	2008-12-31	12
+344	val_344	2008-12-31	12
+344	val_344	2008-12-31	12
+345	val_345	2008-12-31	12
+348	val_348	2008-12-31	12
+348	val_348	2008-12-31	12
+348	val_348	2008-12-31	12
+348	val_348	2008-12-31	12
+348	val_348	2008-12-31	12
+351	val_351	2008-12-31	12
+353	val_353	2008-12-31	12
+353	val_353	2008-12-31	12
+356	val_356	2008-12-31	12
+360	val_360	2008-12-31	12
+362	val_362	2008-12-31	12
+364	val_364	2008-12-31	12
+365	val_365	2008-12-31	12
+366	val_366	2008-12-31	12
+367	val_367	2008-12-31	12
+367	val_367	2008-12-31	12
+368	val_368	2008-12-31	12
+369	val_369	2008-12-31	12
+369	val_369	2008-12-31	12
+369	val_369	2008-12-31	12
+373	val_373	2008-12-31	12
+374	val_374	2008-12-31	12
+375	val_375	2008-12-31	12
+377	val_377	2008-12-31	12
+378	val_378	2008-12-31	12
+379	val_379	2008-12-31	12
+382	val_382	2008-12-31	12
+382	val_382	2008-12-31	12
+384	val_384	2008-12-31	12
+384	val_384	2008-12-31	12
+384	val_384	2008-12-31	12
+386	val_386	2008-12-31	12
+389	val_389	2008-12-31	12
+392	val_392	2008-12-31	12
+393	val_393	2008-12-31	12
+394	val_394	2008-12-31	12
+395	val_395	2008-12-31	12
+395	val_395	2008-12-31	12
+396	val_396	2008-12-31	12
+396	val_396	2008-12-31	12
+396	val_396	2008-12-31	12
+397	val_397	2008-12-31	12
+397	val_397	2008-12-31	12
+399	val_399	2008-12-31	12
+399	val_399	2008-12-31	12
+400	val_400	2008-12-31	12
+401	val_401	2008-12-31	12
+401	val_401	2008-12-31	12
+401	val_401	2008-12-31	12
+401	val_401	2008-12-31	12
+401	val_401	2008-12-31	12
+402	val_402	2008-12-31	12
+403	val_403	2008-12-31	12
+403	val_403	2008-12-31	12
+403	val_403	2008-12-31	12
+404	val_404	2008-12-31	12
+404	val_404	2008-12-31	12
+406	val_406	2008-12-31	12
+406	val_406	2008-12-31	12
+406	val_406	2008-12-31	12
+406	val_406	2008-12-31	12
+407	val_407	2008-12-31	12
+409	val_409	2008-12-31	12
+409	val_409	2008-12-31	12
+409	val_409	2008-12-31	12
+411	val_411	2008-12-31	12
+413	val_413	2008-12-31	12
+413	val_413	2008-12-31	12
+414	val_414	2008-12-31	12
+414	val_414	2008-12-31	12
+417	val_417	2008-12-31	12
+417	val_417	2008-12-31	12
+417	val_417	2008-12-31	12
+418	val_418	2008-12-31	12
+419	val_419	2008-12-31	12
+421	val_421	2008-12-31	12
+424	val_424	2008-12-31	12
+424	val_424	2008-12-31	12
+427	val_427	2008-12-31	12
+429	val_429	2008-12-31	12
+429	val_429	2008-12-31	12
+430	val_430	2008-12-31	12
+430	val_430	2008-12-31	12
+430	val_430	2008-12-31	12
+431	val_431	2008-12-31	12
+431	val_431	2008-12-31	12
+431	val_431	2008-12-31	12
+432	val_432	2008-12-31	12
+435	val_435	2008-12-31	12
+436	val_436	2008-12-31	12
+437	val_437	2008-12-31	12
+438	val_438	2008-12-31	12
+438	val_438	2008-12-31	12
+438	val_438	2008-12-31	12
+439	val_439	2008-12-31	12
+439	val_439	2008-12-31	12
+443	val_443	2008-12-31	12
+444	val_444	2008-12-31	12
+446	val_446	2008-12-31	12
+448	val_448	2008-12-31	12
+449	val_449	2008-12-31	12
+452	val_452	2008-12-31	12
+453	val_453	2008-12-31	12
+454	val_454	2008-12-31	12
+454	val_454	2008-12-31	12
+454	val_454	2008-12-31	12
+455	val_455	2008-12-31	12
+457	val_457	2008-12-31	12
+458	val_458	2008-12-31	12
+458	val_458	2008-12-31	12
+459	val_459	2008-12-31	12
+459	val_459	2008-12-31	12
+460	val_460	2008-12-31	12
+462	val_462	2008-12-31	12
+462	val_462	2008-12-31	12
+463	val_463	2008-12-31	12
+463	val_463	2008-12-31	12
+466	val_466	2008-12-31	12
+466	val_466	2008-12-31	12
+466	val_466	2008-12-31	12
+467	val_467	2008-12-31	12
+468	val_468	2008-12-31	12
+468	val_468	2008-12-31	12
+468	val_468	2008-12-31	12
+468	val_468	2008-12-31	12
+469	val_469	2008-12-31	12
+469	val_469	2008-12-31	12
+469	val_469	2008-12-31	12
+469	val_469	2008-12-31	12
+469	val_469	2008-12-31	12
+470	val_470	2008-12-31	12
+472	val_472	2008-12-31	12
+475	val_475	2008-12-31	12
+477	val_477	2008-12-31	12
+478	val_478	2008-12-31	12
+478	val_478	2008-12-31	12
+479	val_479	2008-12-31	12
+480	val_480	2008-12-31	12
+480	val_480	2008-12-31	12
+480	val_480	2008-12-31	12
+481	val_481	2008-12-31	12
+482	val_482	2008-12-31	12
+483	val_483	2008-12-31	12
+484	val_484	2008-12-31	12
+485	val_485	2008-12-31	12
+487	val_487	2008-12-31	12
+489	val_489	2008-12-31	12
+489	val_489	2008-12-31	12
+489	val_489	2008-12-31	12
+489	val_489	2008-12-31	12
+490	val_490	2008-12-31	12
+491	val_491	2008-12-31	12
+492	val_492	2008-12-31	12
+492	val_492	2008-12-31	12
+493	val_493	2008-12-31	12
+494	val_494	2008-12-31	12
+495	val_495	2008-12-31	12
+496	val_496	2008-12-31	12
+497	val_497	2008-12-31	12
+498	val_498	2008-12-31	12
+498	val_498	2008-12-31	12
+498	val_498	2008-12-31	12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 b/sql/hive/src/test/resources/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531
deleted file mode 100644
index 3994499f925769ee37abee7a6f5f598e6cd05b2b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531
+++ /dev/null
@@ -1,1000 +0,0 @@
-238	val_238	2008-12-31	11
-86	val_86	2008-12-31	11
-311	val_311	2008-12-31	11
-27	val_27	2008-12-31	11
-165	val_165	2008-12-31	11
-409	val_409	2008-12-31	11
-255	val_255	2008-12-31	11
-278	val_278	2008-12-31	11
-98	val_98	2008-12-31	11
-484	val_484	2008-12-31	11
-265	val_265	2008-12-31	11
-193	val_193	2008-12-31	11
-401	val_401	2008-12-31	11
-150	val_150	2008-12-31	11
-273	val_273	2008-12-31	11
-224	val_224	2008-12-31	11
-369	val_369	2008-12-31	11
-66	val_66	2008-12-31	11
-128	val_128	2008-12-31	11
-213	val_213	2008-12-31	11
-146	val_146	2008-12-31	11
-406	val_406	2008-12-31	11
-429	val_429	2008-12-31	11
-374	val_374	2008-12-31	11
-152	val_152	2008-12-31	11
-469	val_469	2008-12-31	11
-145	val_145	2008-12-31	11
-495	val_495	2008-12-31	11
-37	val_37	2008-12-31	11
-327	val_327	2008-12-31	11
-281	val_281	2008-12-31	11
-277	val_277	2008-12-31	11
-209	val_209	2008-12-31	11
-15	val_15	2008-12-31	11
-82	val_82	2008-12-31	11
-403	val_403	2008-12-31	11
-166	val_166	2008-12-31	11
-417	val_417	2008-12-31	11
-430	val_430	2008-12-31	11
-252	val_252	2008-12-31	11
-292	val_292	2008-12-31	11
-219	val_219	2008-12-31	11
-287	val_287	2008-12-31	11
-153	val_153	2008-12-31	11
-193	val_193	2008-12-31	11
-338	val_338	2008-12-31	11
-446	val_446	2008-12-31	11
-459	val_459	2008-12-31	11
-394	val_394	2008-12-31	11
-237	val_237	2008-12-31	11
-482	val_482	2008-12-31	11
-174	val_174	2008-12-31	11
-413	val_413	2008-12-31	11
-494	val_494	2008-12-31	11
-207	val_207	2008-12-31	11
-199	val_199	2008-12-31	11
-466	val_466	2008-12-31	11
-208	val_208	2008-12-31	11
-174	val_174	2008-12-31	11
-399	val_399	2008-12-31	11
-396	val_396	2008-12-31	11
-247	val_247	2008-12-31	11
-417	val_417	2008-12-31	11
-489	val_489	2008-12-31	11
-162	val_162	2008-12-31	11
-377	val_377	2008-12-31	11
-397	val_397	2008-12-31	11
-309	val_309	2008-12-31	11
-365	val_365	2008-12-31	11
-266	val_266	2008-12-31	11
-439	val_439	2008-12-31	11
-342	val_342	2008-12-31	11
-367	val_367	2008-12-31	11
-325	val_325	2008-12-31	11
-167	val_167	2008-12-31	11
-195	val_195	2008-12-31	11
-475	val_475	2008-12-31	11
-17	val_17	2008-12-31	11
-113	val_113	2008-12-31	11
-155	val_155	2008-12-31	11
-203	val_203	2008-12-31	11
-339	val_339	2008-12-31	11
-0	val_0	2008-12-31	11
-455	val_455	2008-12-31	11
-128	val_128	2008-12-31	11
-311	val_311	2008-12-31	11
-316	val_316	2008-12-31	11
-57	val_57	2008-12-31	11
-302	val_302	2008-12-31	11
-205	val_205	2008-12-31	11
-149	val_149	2008-12-31	11
-438	val_438	2008-12-31	11
-345	val_345	2008-12-31	11
-129	val_129	2008-12-31	11
-170	val_170	2008-12-31	11
-20	val_20	2008-12-31	11
-489	val_489	2008-12-31	11
-157	val_157	2008-12-31	11
-378	val_378	2008-12-31	11
-221	val_221	2008-12-31	11
-92	val_92	2008-12-31	11
-111	val_111	2008-12-31	11
-47	val_47	2008-12-31	11
-72	val_72	2008-12-31	11
-4	val_4	2008-12-31	11
-280	val_280	2008-12-31	11
-35	val_35	2008-12-31	11
-427	val_427	2008-12-31	11
-277	val_277	2008-12-31	11
-208	val_208	2008-12-31	11
-356	val_356	2008-12-31	11
-399	val_399	2008-12-31	11
-169	val_169	2008-12-31	11
-382	val_382	2008-12-31	11
-498	val_498	2008-12-31	11
-125	val_125	2008-12-31	11
-386	val_386	2008-12-31	11
-437	val_437	2008-12-31	11
-469	val_469	2008-12-31	11
-192	val_192	2008-12-31	11
-286	val_286	2008-12-31	11
-187	val_187	2008-12-31	11
-176	val_176	2008-12-31	11
-54	val_54	2008-12-31	11
-459	val_459	2008-12-31	11
-51	val_51	2008-12-31	11
-138	val_138	2008-12-31	11
-103	val_103	2008-12-31	11
-239	val_239	2008-12-31	11
-213	val_213	2008-12-31	11
-216	val_216	2008-12-31	11
-430	val_430	2008-12-31	11
-278	val_278	2008-12-31	11
-176	val_176	2008-12-31	11
-289	val_289	2008-12-31	11
-221	val_221	2008-12-31	11
-65	val_65	2008-12-31	11
-318	val_318	2008-12-31	11
-332	val_332	2008-12-31	11
-311	val_311	2008-12-31	11
-275	val_275	2008-12-31	11
-137	val_137	2008-12-31	11
-241	val_241	2008-12-31	11
-83	val_83	2008-12-31	11
-333	val_333	2008-12-31	11
-180	val_180	2008-12-31	11
-284	val_284	2008-12-31	11
-12	val_12	2008-12-31	11
-230	val_230	2008-12-31	11
-181	val_181	2008-12-31	11
-67	val_67	2008-12-31	11
-260	val_260	2008-12-31	11
-404	val_404	2008-12-31	11
-384	val_384	2008-12-31	11
-489	val_489	2008-12-31	11
-353	val_353	2008-12-31	11
-373	val_373	2008-12-31	11
-272	val_272	2008-12-31	11
-138	val_138	2008-12-31	11
-217	val_217	2008-12-31	11
-84	val_84	2008-12-31	11
-348	val_348	2008-12-31	11
-466	val_466	2008-12-31	11
-58	val_58	2008-12-31	11
-8	val_8	2008-12-31	11
-411	val_411	2008-12-31	11
-230	val_230	2008-12-31	11
-208	val_208	2008-12-31	11
-348	val_348	2008-12-31	11
-24	val_24	2008-12-31	11
-463	val_463	2008-12-31	11
-431	val_431	2008-12-31	11
-179	val_179	2008-12-31	11
-172	val_172	2008-12-31	11
-42	val_42	2008-12-31	11
-129	val_129	2008-12-31	11
-158	val_158	2008-12-31	11
-119	val_119	2008-12-31	11
-496	val_496	2008-12-31	11
-0	val_0	2008-12-31	11
-322	val_322	2008-12-31	11
-197	val_197	2008-12-31	11
-468	val_468	2008-12-31	11
-393	val_393	2008-12-31	11
-454	val_454	2008-12-31	11
-100	val_100	2008-12-31	11
-298	val_298	2008-12-31	11
-199	val_199	2008-12-31	11
-191	val_191	2008-12-31	11
-418	val_418	2008-12-31	11
-96	val_96	2008-12-31	11
-26	val_26	2008-12-31	11
-165	val_165	2008-12-31	11
-327	val_327	2008-12-31	11
-230	val_230	2008-12-31	11
-205	val_205	2008-12-31	11
-120	val_120	2008-12-31	11
-131	val_131	2008-12-31	11
-51	val_51	2008-12-31	11
-404	val_404	2008-12-31	11
-43	val_43	2008-12-31	11
-436	val_436	2008-12-31	11
-156	val_156	2008-12-31	11
-469	val_469	2008-12-31	11
-468	val_468	2008-12-31	11
-308	val_308	2008-12-31	11
-95	val_95	2008-12-31	11
-196	val_196	2008-12-31	11
-288	val_288	2008-12-31	11
-481	val_481	2008-12-31	11
-457	val_457	2008-12-31	11
-98	val_98	2008-12-31	11
-282	val_282	2008-12-31	11
-197	val_197	2008-12-31	11
-187	val_187	2008-12-31	11
-318	val_318	2008-12-31	11
-318	val_318	2008-12-31	11
-409	val_409	2008-12-31	11
-470	val_470	2008-12-31	11
-137	val_137	2008-12-31	11
-369	val_369	2008-12-31	11
-316	val_316	2008-12-31	11
-169	val_169	2008-12-31	11
-413	val_413	2008-12-31	11
-85	val_85	2008-12-31	11
-77	val_77	2008-12-31	11
-0	val_0	2008-12-31	11
-490	val_490	2008-12-31	11
-87	val_87	2008-12-31	11
-364	val_364	2008-12-31	11
-179	val_179	2008-12-31	11
-118	val_118	2008-12-31	11
-134	val_134	2008-12-31	11
-395	val_395	2008-12-31	11
-282	val_282	2008-12-31	11
-138	val_138	2008-12-31	11
-238	val_238	2008-12-31	11
-419	val_419	2008-12-31	11
-15	val_15	2008-12-31	11
-118	val_118	2008-12-31	11
-72	val_72	2008-12-31	11
-90	val_90	2008-12-31	11
-307	val_307	2008-12-31	11
-19	val_19	2008-12-31	11
-435	val_435	2008-12-31	11
-10	val_10	2008-12-31	11
-277	val_277	2008-12-31	11
-273	val_273	2008-12-31	11
-306	val_306	2008-12-31	11
-224	val_224	2008-12-31	11
-309	val_309	2008-12-31	11
-389	val_389	2008-12-31	11
-327	val_327	2008-12-31	11
-242	val_242	2008-12-31	11
-369	val_369	2008-12-31	11
-392	val_392	2008-12-31	11
-272	val_272	2008-12-31	11
-331	val_331	2008-12-31	11
-401	val_401	2008-12-31	11
-242	val_242	2008-12-31	11
-452	val_452	2008-12-31	11
-177	val_177	2008-12-31	11
-226	val_226	2008-12-31	11
-5	val_5	2008-12-31	11
-497	val_497	2008-12-31	11
-402	val_402	2008-12-31	11
-396	val_396	2008-12-31	11
-317	val_317	2008-12-31	11
-395	val_395	2008-12-31	11
-58	val_58	2008-12-31	11
-35	val_35	2008-12-31	11
-336	val_336	2008-12-31	11
-95	val_95	2008-12-31	11
-11	val_11	2008-12-31	11
-168	val_168	2008-12-31	11
-34	val_34	2008-12-31	11
-229	val_229	2008-12-31	11
-233	val_233	2008-12-31	11
-143	val_143	2008-12-31	11
-472	val_472	2008-12-31	11
-322	val_322	2008-12-31	11
-498	val_498	2008-12-31	11
-160	val_160	2008-12-31	11
-195	val_195	2008-12-31	11
-42	val_42	2008-12-31	11
-321	val_321	2008-12-31	11
-430	val_430	2008-12-31	11
-119	val_119	2008-12-31	11
-489	val_489	2008-12-31	11
-458	val_458	2008-12-31	11
-78	val_78	2008-12-31	11
-76	val_76	2008-12-31	11
-41	val_41	2008-12-31	11
-223	val_223	2008-12-31	11
-492	val_492	2008-12-31	11
-149	val_149	2008-12-31	11
-449	val_449	2008-12-31	11
-218	val_218	2008-12-31	11
-228	val_228	2008-12-31	11
-138	val_138	2008-12-31	11
-453	val_453	2008-12-31	11
-30	val_30	2008-12-31	11
-209	val_209	2008-12-31	11
-64	val_64	2008-12-31	11
-468	val_468	2008-12-31	11
-76	val_76	2008-12-31	11
-74	val_74	2008-12-31	11
-342	val_342	2008-12-31	11
-69	val_69	2008-12-31	11
-230	val_230	2008-12-31	11
-33	val_33	2008-12-31	11
-368	val_368	2008-12-31	11
-103	val_103	2008-12-31	11
-296	val_296	2008-12-31	11
-113	val_113	2008-12-31	11
-216	val_216	2008-12-31	11
-367	val_367	2008-12-31	11
-344	val_344	2008-12-31	11
-167	val_167	2008-12-31	11
-274	val_274	2008-12-31	11
-219	val_219	2008-12-31	11
-239	val_239	2008-12-31	11
-485	val_485	2008-12-31	11
-116	val_116	2008-12-31	11
-223	val_223	2008-12-31	11
-256	val_256	2008-12-31	11
-263	val_263	2008-12-31	11
-70	val_70	2008-12-31	11
-487	val_487	2008-12-31	11
-480	val_480	2008-12-31	11
-401	val_401	2008-12-31	11
-288	val_288	2008-12-31	11
-191	val_191	2008-12-31	11
-5	val_5	2008-12-31	11
-244	val_244	2008-12-31	11
-438	val_438	2008-12-31	11
-128	val_128	2008-12-31	11
-467	val_467	2008-12-31	11
-432	val_432	2008-12-31	11
-202	val_202	2008-12-31	11
-316	val_316	2008-12-31	11
-229	val_229	2008-12-31	11
-469	val_469	2008-12-31	11
-463	val_463	2008-12-31	11
-280	val_280	2008-12-31	11
-2	val_2	2008-12-31	11
-35	val_35	2008-12-31	11
-283	val_283	2008-12-31	11
-331	val_331	2008-12-31	11
-235	val_235	2008-12-31	11
-80	val_80	2008-12-31	11
-44	val_44	2008-12-31	11
-193	val_193	2008-12-31	11
-321	val_321	2008-12-31	11
-335	val_335	2008-12-31	11
-104	val_104	2008-12-31	11
-466	val_466	2008-12-31	11
-366	val_366	2008-12-31	11
-175	val_175	2008-12-31	11
-403	val_403	2008-12-31	11
-483	val_483	2008-12-31	11
-53	val_53	2008-12-31	11
-105	val_105	2008-12-31	11
-257	val_257	2008-12-31	11
-406	val_406	2008-12-31	11
-409	val_409	2008-12-31	11
-190	val_190	2008-12-31	11
-406	val_406	2008-12-31	11
-401	val_401	2008-12-31	11
-114	val_114	2008-12-31	11
-258	val_258	2008-12-31	11
-90	val_90	2008-12-31	11
-203	val_203	2008-12-31	11
-262	val_262	2008-12-31	11
-348	val_348	2008-12-31	11
-424	val_424	2008-12-31	11
-12	val_12	2008-12-31	11
-396	val_396	2008-12-31	11
-201	val_201	2008-12-31	11
-217	val_217	2008-12-31	11
-164	val_164	2008-12-31	11
-431	val_431	2008-12-31	11
-454	val_454	2008-12-31	11
-478	val_478	2008-12-31	11
-298	val_298	2008-12-31	11
-125	val_125	2008-12-31	11
-431	val_431	2008-12-31	11
-164	val_164	2008-12-31	11
-424	val_424	2008-12-31	11
-187	val_187	2008-12-31	11
-382	val_382	2008-12-31	11
-5	val_5	2008-12-31	11
-70	val_70	2008-12-31	11
-397	val_397	2008-12-31	11
-480	val_480	2008-12-31	11
-291	val_291	2008-12-31	11
-24	val_24	2008-12-31	11
-351	val_351	2008-12-31	11
-255	val_255	2008-12-31	11
-104	val_104	2008-12-31	11
-70	val_70	2008-12-31	11
-163	val_163	2008-12-31	11
-438	val_438	2008-12-31	11
-119	val_119	2008-12-31	11
-414	val_414	2008-12-31	11
-200	val_200	2008-12-31	11
-491	val_491	2008-12-31	11
-237	val_237	2008-12-31	11
-439	val_439	2008-12-31	11
-360	val_360	2008-12-31	11
-248	val_248	2008-12-31	11
-479	val_479	2008-12-31	11
-305	val_305	2008-12-31	11
-417	val_417	2008-12-31	11
-199	val_199	2008-12-31	11
-444	val_444	2008-12-31	11
-120	val_120	2008-12-31	11
-429	val_429	2008-12-31	11
-169	val_169	2008-12-31	11
-443	val_443	2008-12-31	11
-323	val_323	2008-12-31	11
-325	val_325	2008-12-31	11
-277	val_277	2008-12-31	11
-230	val_230	2008-12-31	11
-478	val_478	2008-12-31	11
-178	val_178	2008-12-31	11
-468	val_468	2008-12-31	11
-310	val_310	2008-12-31	11
-317	val_317	2008-12-31	11
-333	val_333	2008-12-31	11
-493	val_493	2008-12-31	11
-460	val_460	2008-12-31	11
-207	val_207	2008-12-31	11
-249	val_249	2008-12-31	11
-265	val_265	2008-12-31	11
-480	val_480	2008-12-31	11
-83	val_83	2008-12-31	11
-136	val_136	2008-12-31	11
-353	val_353	2008-12-31	11
-172	val_172	2008-12-31	11
-214	val_214	2008-12-31	11
-462	val_462	2008-12-31	11
-233	val_233	2008-12-31	11
-406	val_406	2008-12-31	11
-133	val_133	2008-12-31	11
-175	val_175	2008-12-31	11
-189	val_189	2008-12-31	11
-454	val_454	2008-12-31	11
-375	val_375	2008-12-31	11
-401	val_401	2008-12-31	11
-421	val_421	2008-12-31	11
-407	val_407	2008-12-31	11
-384	val_384	2008-12-31	11
-256	val_256	2008-12-31	11
-26	val_26	2008-12-31	11
-134	val_134	2008-12-31	11
-67	val_67	2008-12-31	11
-384	val_384	2008-12-31	11
-379	val_379	2008-12-31	11
-18	val_18	2008-12-31	11
-462	val_462	2008-12-31	11
-492	val_492	2008-12-31	11
-100	val_100	2008-12-31	11
-298	val_298	2008-12-31	11
-9	val_9	2008-12-31	11
-341	val_341	2008-12-31	11
-498	val_498	2008-12-31	11
-146	val_146	2008-12-31	11
-458	val_458	2008-12-31	11
-362	val_362	2008-12-31	11
-186	val_186	2008-12-31	11
-285	val_285	2008-12-31	11
-348	val_348	2008-12-31	11
-167	val_167	2008-12-31	11
-18	val_18	2008-12-31	11
-273	val_273	2008-12-31	11
-183	val_183	2008-12-31	11
-281	val_281	2008-12-31	11
-344	val_344	2008-12-31	11
-97	val_97	2008-12-31	11
-469	val_469	2008-12-31	11
-315	val_315	2008-12-31	11
-84	val_84	2008-12-31	11
-28	val_28	2008-12-31	11
-37	val_37	2008-12-31	11
-448	val_448	2008-12-31	11
-152	val_152	2008-12-31	11
-348	val_348	2008-12-31	11
-307	val_307	2008-12-31	11
-194	val_194	2008-12-31	11
-414	val_414	2008-12-31	11
-477	val_477	2008-12-31	11
-222	val_222	2008-12-31	11
-126	val_126	2008-12-31	11
-90	val_90	2008-12-31	11
-169	val_169	2008-12-31	11
-403	val_403	2008-12-31	11
-400	val_400	2008-12-31	11
-200	val_200	2008-12-31	11
-97	val_97	2008-12-31	11
-238	val_238	2008-12-31	12
-86	val_86	2008-12-31	12
-311	val_311	2008-12-31	12
-27	val_27	2008-12-31	12
-165	val_165	2008-12-31	12
-409	val_409	2008-12-31	12
-255	val_255	2008-12-31	12
-278	val_278	2008-12-31	12
-98	val_98	2008-12-31	12
-484	val_484	2008-12-31	12
-265	val_265	2008-12-31	12
-193	val_193	2008-12-31	12
-401	val_401	2008-12-31	12
-150	val_150	2008-12-31	12
-273	val_273	2008-12-31	12
-224	val_224	2008-12-31	12
-369	val_369	2008-12-31	12
-66	val_66	2008-12-31	12
-128	val_128	2008-12-31	12
-213	val_213	2008-12-31	12
-146	val_146	2008-12-31	12
-406	val_406	2008-12-31	12
-429	val_429	2008-12-31	12
-374	val_374	2008-12-31	12
-152	val_152	2008-12-31	12
-469	val_469	2008-12-31	12
-145	val_145	2008-12-31	12
-495	val_495	2008-12-31	12
-37	val_37	2008-12-31	12
-327	val_327	2008-12-31	12
-281	val_281	2008-12-31	12
-277	val_277	2008-12-31	12
-209	val_209	2008-12-31	12
-15	val_15	2008-12-31	12
-82	val_82	2008-12-31	12
-403	val_403	2008-12-31	12
-166	val_166	2008-12-31	12
-417	val_417	2008-12-31	12
-430	val_430	2008-12-31	12
-252	val_252	2008-12-31	12
-292	val_292	2008-12-31	12
-219	val_219	2008-12-31	12
-287	val_287	2008-12-31	12
-153	val_153	2008-12-31	12
-193	val_193	2008-12-31	12
-338	val_338	2008-12-31	12
-446	val_446	2008-12-31	12
-459	val_459	2008-12-31	12
-394	val_394	2008-12-31	12
-237	val_237	2008-12-31	12
-482	val_482	2008-12-31	12
-174	val_174	2008-12-31	12
-413	val_413	2008-12-31	12
-494	val_494	2008-12-31	12
-207	val_207	2008-12-31	12
-199	val_199	2008-12-31	12
-466	val_466	2008-12-31	12
-208	val_208	2008-12-31	12
-174	val_174	2008-12-31	12
-399	val_399	2008-12-31	12
-396	val_396	2008-12-31	12
-247	val_247	2008-12-31	12
-417	val_417	2008-12-31	12
-489	val_489	2008-12-31	12
-162	val_162	2008-12-31	12
-377	val_377	2008-12-31	12
-397	val_397	2008-12-31	12
-309	val_309	2008-12-31	12
-365	val_365	2008-12-31	12
-266	val_266	2008-12-31	12
-439	val_439	2008-12-31	12
-342	val_342	2008-12-31	12
-367	val_367	2008-12-31	12
-325	val_325	2008-12-31	12
-167	val_167	2008-12-31	12
-195	val_195	2008-12-31	12
-475	val_475	2008-12-31	12
-17	val_17	2008-12-31	12
-113	val_113	2008-12-31	12
-155	val_155	2008-12-31	12
-203	val_203	2008-12-31	12
-339	val_339	2008-12-31	12
-0	val_0	2008-12-31	12
-455	val_455	2008-12-31	12
-128	val_128	2008-12-31	12
-311	val_311	2008-12-31	12
-316	val_316	2008-12-31	12
-57	val_57	2008-12-31	12
-302	val_302	2008-12-31	12
-205	val_205	2008-12-31	12
-149	val_149	2008-12-31	12
-438	val_438	2008-12-31	12
-345	val_345	2008-12-31	12
-129	val_129	2008-12-31	12
-170	val_170	2008-12-31	12
-20	val_20	2008-12-31	12
-489	val_489	2008-12-31	12
-157	val_157	2008-12-31	12
-378	val_378	2008-12-31	12
-221	val_221	2008-12-31	12
-92	val_92	2008-12-31	12
-111	val_111	2008-12-31	12
-47	val_47	2008-12-31	12
-72	val_72	2008-12-31	12
-4	val_4	2008-12-31	12
-280	val_280	2008-12-31	12
-35	val_35	2008-12-31	12
-427	val_427	2008-12-31	12
-277	val_277	2008-12-31	12
-208	val_208	2008-12-31	12
-356	val_356	2008-12-31	12
-399	val_399	2008-12-31	12
-169	val_169	2008-12-31	12
-382	val_382	2008-12-31	12
-498	val_498	2008-12-31	12
-125	val_125	2008-12-31	12
-386	val_386	2008-12-31	12
-437	val_437	2008-12-31	12
-469	val_469	2008-12-31	12
-192	val_192	2008-12-31	12
-286	val_286	2008-12-31	12
-187	val_187	2008-12-31	12
-176	val_176	2008-12-31	12
-54	val_54	2008-12-31	12
-459	val_459	2008-12-31	12
-51	val_51	2008-12-31	12
-138	val_138	2008-12-31	12
-103	val_103	2008-12-31	12
-239	val_239	2008-12-31	12
-213	val_213	2008-12-31	12
-216	val_216	2008-12-31	12
-430	val_430	2008-12-31	12
-278	val_278	2008-12-31	12
-176	val_176	2008-12-31	12
-289	val_289	2008-12-31	12
-221	val_221	2008-12-31	12
-65	val_65	2008-12-31	12
-318	val_318	2008-12-31	12
-332	val_332	2008-12-31	12
-311	val_311	2008-12-31	12
-275	val_275	2008-12-31	12
-137	val_137	2008-12-31	12
-241	val_241	2008-12-31	12
-83	val_83	2008-12-31	12
-333	val_333	2008-12-31	12
-180	val_180	2008-12-31	12
-284	val_284	2008-12-31	12
-12	val_12	2008-12-31	12
-230	val_230	2008-12-31	12
-181	val_181	2008-12-31	12
-67	val_67	2008-12-31	12
-260	val_260	2008-12-31	12
-404	val_404	2008-12-31	12
-384	val_384	2008-12-31	12
-489	val_489	2008-12-31	12
-353	val_353	2008-12-31	12
-373	val_373	2008-12-31	12
-272	val_272	2008-12-31	12
-138	val_138	2008-12-31	12
-217	val_217	2008-12-31	12
-84	val_84	2008-12-31	12
-348	val_348	2008-12-31	12
-466	val_466	2008-12-31	12
-58	val_58	2008-12-31	12
-8	val_8	2008-12-31	12
-411	val_411	2008-12-31	12
-230	val_230	2008-12-31	12
-208	val_208	2008-12-31	12
-348	val_348	2008-12-31	12
-24	val_24	2008-12-31	12
-463	val_463	2008-12-31	12
-431	val_431	2008-12-31	12
-179	val_179	2008-12-31	12
-172	val_172	2008-12-31	12
-42	val_42	2008-12-31	12
-129	val_129	2008-12-31	12
-158	val_158	2008-12-31	12
-119	val_119	2008-12-31	12
-496	val_496	2008-12-31	12
-0	val_0	2008-12-31	12
-322	val_322	2008-12-31	12
-197	val_197	2008-12-31	12
-468	val_468	2008-12-31	12
-393	val_393	2008-12-31	12
-454	val_454	2008-12-31	12
-100	val_100	2008-12-31	12
-298	val_298	2008-12-31	12
-199	val_199	2008-12-31	12
-191	val_191	2008-12-31	12
-418	val_418	2008-12-31	12
-96	val_96	2008-12-31	12
-26	val_26	2008-12-31	12
-165	val_165	2008-12-31	12
-327	val_327	2008-12-31	12
-230	val_230	2008-12-31	12
-205	val_205	2008-12-31	12
-120	val_120	2008-12-31	12
-131	val_131	2008-12-31	12
-51	val_51	2008-12-31	12
-404	val_404	2008-12-31	12
-43	val_43	2008-12-31	12
-436	val_436	2008-12-31	12
-156	val_156	2008-12-31	12
-469	val_469	2008-12-31	12
-468	val_468	2008-12-31	12
-308	val_308	2008-12-31	12
-95	val_95	2008-12-31	12
-196	val_196	2008-12-31	12
-288	val_288	2008-12-31	12
-481	val_481	2008-12-31	12
-457	val_457	2008-12-31	12
-98	val_98	2008-12-31	12
-282	val_282	2008-12-31	12
-197	val_197	2008-12-31	12
-187	val_187	2008-12-31	12
-318	val_318	2008-12-31	12
-318	val_318	2008-12-31	12
-409	val_409	2008-12-31	12
-470	val_470	2008-12-31	12
-137	val_137	2008-12-31	12
-369	val_369	2008-12-31	12
-316	val_316	2008-12-31	12
-169	val_169	2008-12-31	12
-413	val_413	2008-12-31	12
-85	val_85	2008-12-31	12
-77	val_77	2008-12-31	12
-0	val_0	2008-12-31	12
-490	val_490	2008-12-31	12
-87	val_87	2008-12-31	12
-364	val_364	2008-12-31	12
-179	val_179	2008-12-31	12
-118	val_118	2008-12-31	12
-134	val_134	2008-12-31	12
-395	val_395	2008-12-31	12
-282	val_282	2008-12-31	12
-138	val_138	2008-12-31	12
-238	val_238	2008-12-31	12
-419	val_419	2008-12-31	12
-15	val_15	2008-12-31	12
-118	val_118	2008-12-31	12
-72	val_72	2008-12-31	12
-90	val_90	2008-12-31	12
-307	val_307	2008-12-31	12
-19	val_19	2008-12-31	12
-435	val_435	2008-12-31	12
-10	val_10	2008-12-31	12
-277	val_277	2008-12-31	12
-273	val_273	2008-12-31	12
-306	val_306	2008-12-31	12
-224	val_224	2008-12-31	12
-309	val_309	2008-12-31	12
-389	val_389	2008-12-31	12
-327	val_327	2008-12-31	12
-242	val_242	2008-12-31	12
-369	val_369	2008-12-31	12
-392	val_392	2008-12-31	12
-272	val_272	2008-12-31	12
-331	val_331	2008-12-31	12
-401	val_401	2008-12-31	12
-242	val_242	2008-12-31	12
-452	val_452	2008-12-31	12
-177	val_177	2008-12-31	12
-226	val_226	2008-12-31	12
-5	val_5	2008-12-31	12
-497	val_497	2008-12-31	12
-402	val_402	2008-12-31	12
-396	val_396	2008-12-31	12
-317	val_317	2008-12-31	12
-395	val_395	2008-12-31	12
-58	val_58	2008-12-31	12
-35	val_35	2008-12-31	12
-336	val_336	2008-12-31	12
-95	val_95	2008-12-31	12
-11	val_11	2008-12-31	12
-168	val_168	2008-12-31	12
-34	val_34	2008-12-31	12
-229	val_229	2008-12-31	12
-233	val_233	2008-12-31	12
-143	val_143	2008-12-31	12
-472	val_472	2008-12-31	12
-322	val_322	2008-12-31	12
-498	val_498	2008-12-31	12
-160	val_160	2008-12-31	12
-195	val_195	2008-12-31	12
-42	val_42	2008-12-31	12
-321	val_321	2008-12-31	12
-430	val_430	2008-12-31	12
-119	val_119	2008-12-31	12
-489	val_489	2008-12-31	12
-458	val_458	2008-12-31	12
-78	val_78	2008-12-31	12
-76	val_76	2008-12-31	12
-41	val_41	2008-12-31	12
-223	val_223	2008-12-31	12
-492	val_492	2008-12-31	12
-149	val_149	2008-12-31	12
-449	val_449	2008-12-31	12
-218	val_218	2008-12-31	12
-228	val_228	2008-12-31	12
-138	val_138	2008-12-31	12
-453	val_453	2008-12-31	12
-30	val_30	2008-12-31	12
-209	val_209	2008-12-31	12
-64	val_64	2008-12-31	12
-468	val_468	2008-12-31	12
-76	val_76	2008-12-31	12
-74	val_74	2008-12-31	12
-342	val_342	2008-12-31	12
-69	val_69	2008-12-31	12
-230	val_230	2008-12-31	12
-33	val_33	2008-12-31	12
-368	val_368	2008-12-31	12
-103	val_103	2008-12-31	12
-296	val_296	2008-12-31	12
-113	val_113	2008-12-31	12
-216	val_216	2008-12-31	12
-367	val_367	2008-12-31	12
-344	val_344	2008-12-31	12
-167	val_167	2008-12-31	12
-274	val_274	2008-12-31	12
-219	val_219	2008-12-31	12
-239	val_239	2008-12-31	12
-485	val_485	2008-12-31	12
-116	val_116	2008-12-31	12
-223	val_223	2008-12-31	12
-256	val_256	2008-12-31	12
-263	val_263	2008-12-31	12
-70	val_70	2008-12-31	12
-487	val_487	2008-12-31	12
-480	val_480	2008-12-31	12
-401	val_401	2008-12-31	12
-288	val_288	2008-12-31	12
-191	val_191	2008-12-31	12
-5	val_5	2008-12-31	12
-244	val_244	2008-12-31	12
-438	val_438	2008-12-31	12
-128	val_128	2008-12-31	12
-467	val_467	2008-12-31	12
-432	val_432	2008-12-31	12
-202	val_202	2008-12-31	12
-316	val_316	2008-12-31	12
-229	val_229	2008-12-31	12
-469	val_469	2008-12-31	12
-463	val_463	2008-12-31	12
-280	val_280	2008-12-31	12
-2	val_2	2008-12-31	12
-35	val_35	2008-12-31	12
-283	val_283	2008-12-31	12
-331	val_331	2008-12-31	12
-235	val_235	2008-12-31	12
-80	val_80	2008-12-31	12
-44	val_44	2008-12-31	12
-193	val_193	2008-12-31	12
-321	val_321	2008-12-31	12
-335	val_335	2008-12-31	12
-104	val_104	2008-12-31	12
-466	val_466	2008-12-31	12
-366	val_366	2008-12-31	12
-175	val_175	2008-12-31	12
-403	val_403	2008-12-31	12
-483	val_483	2008-12-31	12
-53	val_53	2008-12-31	12
-105	val_105	2008-12-31	12
-257	val_257	2008-12-31	12
-406	val_406	2008-12-31	12
-409	val_409	2008-12-31	12
-190	val_190	2008-12-31	12
-406	val_406	2008-12-31	12
-401	val_401	2008-12-31	12
-114	val_114	2008-12-31	12
-258	val_258	2008-12-31	12
-90	val_90	2008-12-31	12
-203	val_203	2008-12-31	12
-262	val_262	2008-12-31	12
-348	val_348	2008-12-31	12
-424	val_424	2008-12-31	12
-12	val_12	2008-12-31	12
-396	val_396	2008-12-31	12
-201	val_201	2008-12-31	12
-217	val_217	2008-12-31	12
-164	val_164	2008-12-31	12
-431	val_431	2008-12-31	12
-454	val_454	2008-12-31	12
-478	val_478	2008-12-31	12
-298	val_298	2008-12-31	12
-125	val_125	2008-12-31	12
-431	val_431	2008-12-31	12
-164	val_164	2008-12-31	12
-424	val_424	2008-12-31	12
-187	val_187	2008-12-31	12
-382	val_382	2008-12-31	12
-5	val_5	2008-12-31	12
-70	val_70	2008-12-31	12
-397	val_397	2008-12-31	12
-480	val_480	2008-12-31	12
-291	val_291	2008-12-31	12
-24	val_24	2008-12-31	12
-351	val_351	2008-12-31	12
-255	val_255	2008-12-31	12
-104	val_104	2008-12-31	12
-70	val_70	2008-12-31	12
-163	val_163	2008-12-31	12
-438	val_438	2008-12-31	12
-119	val_119	2008-12-31	12
-414	val_414	2008-12-31	12
-200	val_200	2008-12-31	12
-491	val_491	2008-12-31	12
-237	val_237	2008-12-31	12
-439	val_439	2008-12-31	12
-360	val_360	2008-12-31	12
-248	val_248	2008-12-31	12
-479	val_479	2008-12-31	12
-305	val_305	2008-12-31	12
-417	val_417	2008-12-31	12
-199	val_199	2008-12-31	12
-444	val_444	2008-12-31	12
-120	val_120	2008-12-31	12
-429	val_429	2008-12-31	12
-169	val_169	2008-12-31	12
-443	val_443	2008-12-31	12
-323	val_323	2008-12-31	12
-325	val_325	2008-12-31	12
-277	val_277	2008-12-31	12
-230	val_230	2008-12-31	12
-478	val_478	2008-12-31	12
-178	val_178	2008-12-31	12
-468	val_468	2008-12-31	12
-310	val_310	2008-12-31	12
-317	val_317	2008-12-31	12
-333	val_333	2008-12-31	12
-493	val_493	2008-12-31	12
-460	val_460	2008-12-31	12
-207	val_207	2008-12-31	12
-249	val_249	2008-12-31	12
-265	val_265	2008-12-31	12
-480	val_480	2008-12-31	12
-83	val_83	2008-12-31	12
-136	val_136	2008-12-31	12
-353	val_353	2008-12-31	12
-172	val_172	2008-12-31	12
-214	val_214	2008-12-31	12
-462	val_462	2008-12-31	12
-233	val_233	2008-12-31	12
-406	val_406	2008-12-31	12
-133	val_133	2008-12-31	12
-175	val_175	2008-12-31	12
-189	val_189	2008-12-31	12
-454	val_454	2008-12-31	12
-375	val_375	2008-12-31	12
-401	val_401	2008-12-31	12
-421	val_421	2008-12-31	12
-407	val_407	2008-12-31	12
-384	val_384	2008-12-31	12
-256	val_256	2008-12-31	12
-26	val_26	2008-12-31	12
-134	val_134	2008-12-31	12
-67	val_67	2008-12-31	12
-384	val_384	2008-12-31	12
-379	val_379	2008-12-31	12
-18	val_18	2008-12-31	12
-462	val_462	2008-12-31	12
-492	val_492	2008-12-31	12
-100	val_100	2008-12-31	12
-298	val_298	2008-12-31	12
-9	val_9	2008-12-31	12
-341	val_341	2008-12-31	12
-498	val_498	2008-12-31	12
-146	val_146	2008-12-31	12
-458	val_458	2008-12-31	12
-362	val_362	2008-12-31	12
-186	val_186	2008-12-31	12
-285	val_285	2008-12-31	12
-348	val_348	2008-12-31	12
-167	val_167	2008-12-31	12
-18	val_18	2008-12-31	12
-273	val_273	2008-12-31	12
-183	val_183	2008-12-31	12
-281	val_281	2008-12-31	12
-344	val_344	2008-12-31	12
-97	val_97	2008-12-31	12
-469	val_469	2008-12-31	12
-315	val_315	2008-12-31	12
-84	val_84	2008-12-31	12
-28	val_28	2008-12-31	12
-37	val_37	2008-12-31	12
-448	val_448	2008-12-31	12
-152	val_152	2008-12-31	12
-348	val_348	2008-12-31	12
-307	val_307	2008-12-31	12
-194	val_194	2008-12-31	12
-414	val_414	2008-12-31	12
-477	val_477	2008-12-31	12
-222	val_222	2008-12-31	12
-126	val_126	2008-12-31	12
-90	val_90	2008-12-31	12
-169	val_169	2008-12-31	12
-403	val_403	2008-12-31	12
-400	val_400	2008-12-31	12
-200	val_200	2008-12-31	12
-97	val_97	2008-12-31	12
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302
index e9c723bbd136e561e0ff75d4b248e96ccbf81e73..8c43153cf66f9f265bbe39da3810288e57771358 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302
+++ b/sql/hive/src/test/resources/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302
@@ -1,4 +1,4 @@
 ds=2008-04-08/hr=11
 ds=2008-04-08/hr=12
 ds=2008-04-09/hr=11
-ds=2008-04-09/hr=12
\ No newline at end of file
+ds=2008-04-09/hr=12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2 b/sql/hive/src/test/resources/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2
index 6aa67737fa92db1d02e439f97a9d26e75abe5a8e..36add9b0fc2cef84ae44f071051456ed42ff82c8 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2
+++ b/sql/hive/src/test/resources/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2
@@ -1,12 +1,12 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-hr                  	string              	None                
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part, dbName:default, owner:marmbrus, createTime:1389738838, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2847673635801001933/nzhang_part, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1389738838}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:nzhang_part, dbName:default, owner:marmbrus, createTime:1413887427, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887427}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab
+++ b/sql/hive/src/test/resources/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d b/sql/hive/src/test/resources/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d
+++ b/sql/hive/src/test/resources/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7
+++ b/sql/hive/src/test/resources/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb b/sql/hive/src/test/resources/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb
index 29ade2aff0542574b6fc70b35852c840b1c48e79..7df5f90186db3b5365f76fcf6b227c910acfa195 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb
+++ b/sql/hive/src/test/resources/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb
@@ -997,4 +997,4 @@
 403	val_403	2010-03-03	11
 400	val_400	2010-03-03	11
 200	val_200	2010-03-03	11
-97	val_97	2010-03-03	11
\ No newline at end of file
+97	val_97	2010-03-03	11
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a b/sql/hive/src/test/resources/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a
index b7b1b914db85736c70aa4bfce112312b5bc49bcd..c869646753baade7ffbc76ab1cdcb938d9029cce 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a
+++ b/sql/hive/src/test/resources/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a
@@ -997,4 +997,4 @@
 403	val_403	2010-03-03	12
 400	val_400	2010-03-03	12
 200	val_200	2010-03-03	12
-97	val_97	2010-03-03	12
\ No newline at end of file
+97	val_97	2010-03-03	12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302
index e9c723bbd136e561e0ff75d4b248e96ccbf81e73..8c43153cf66f9f265bbe39da3810288e57771358 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302
+++ b/sql/hive/src/test/resources/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302
@@ -1,4 +1,4 @@
 ds=2008-04-08/hr=11
 ds=2008-04-08/hr=12
 ds=2008-04-09/hr=11
-ds=2008-04-09/hr=12
\ No newline at end of file
+ds=2008-04-09/hr=12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b b/sql/hive/src/test/resources/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b
index 0c283c5378a6f06d19b7a9127dd74d702866fef7..d42b4efb2cac8539592b69f00b91d45d92e2793a 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b
+++ b/sql/hive/src/test/resources/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b
@@ -1,12 +1,12 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-hr                  	string              	None                
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part12, dbName:default, owner:marmbrus, createTime:1389738821, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1431818582215388621/nzhang_part12, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1389738821}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:nzhang_part12, dbName:default, owner:marmbrus, createTime:1413887435, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part12, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887435}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab
+++ b/sql/hive/src/test/resources/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7
+++ b/sql/hive/src/test/resources/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 b/sql/hive/src/test/resources/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97
index 8eb67466a7a895d3bea992b4803d9183b3a7c290..9c35fe7429150bcd6346818e70ab7d4441c25719 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97
+++ b/sql/hive/src/test/resources/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97
@@ -1,2 +1,2 @@
 ds=2010-03-03/hr=22
-ds=2010-03-03/hr=24
\ No newline at end of file
+ds=2010-03-03/hr=24
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345 b/sql/hive/src/test/resources/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345
index 9f50245c181776cf542c6f785e90fdd43744128c..861dc02d49d022aeae1c3b494f20ed3b2ebc7afb 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345
+++ b/sql/hive/src/test/resources/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345
@@ -1997,4 +1997,4 @@
 403	val_403	2010-03-03	24
 400	val_400	2010-03-03	24
 200	val_200	2010-03-03	24
-97	val_97	2010-03-03	24
\ No newline at end of file
+97	val_97	2010-03-03	24
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302
index e9c723bbd136e561e0ff75d4b248e96ccbf81e73..8c43153cf66f9f265bbe39da3810288e57771358 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302
+++ b/sql/hive/src/test/resources/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302
@@ -1,4 +1,4 @@
 ds=2008-04-08/hr=11
 ds=2008-04-08/hr=12
 ds=2008-04-09/hr=11
-ds=2008-04-09/hr=12
\ No newline at end of file
+ds=2008-04-09/hr=12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0 b/sql/hive/src/test/resources/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0
index 08699f286e384d1872c18498385ac02b6346db11..06a16f6477cfa92fd072fa9879cd915978bc5a24 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0
+++ b/sql/hive/src/test/resources/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0
@@ -1,12 +1,12 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-hr                  	string              	None                
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part13, dbName:default, owner:marmbrus, createTime:1389739606, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5344690580869150883/nzhang_part13, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1389739606}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:nzhang_part13, dbName:default, owner:marmbrus, createTime:1413887445, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part13, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887445}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab
+++ b/sql/hive/src/test/resources/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7
+++ b/sql/hive/src/test/resources/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16 b/sql/hive/src/test/resources/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16
index dfe9bcc93bcd3f4625444486de436aa35d19728f..c00f33327744a7a81a7ce3be732fafa54046b169 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16
+++ b/sql/hive/src/test/resources/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16
@@ -1,2 +1,2 @@
 ds=2010-03-03/hr=22
-ds=2010-03-03/hr=33
\ No newline at end of file
+ds=2010-03-03/hr=33
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d b/sql/hive/src/test/resources/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d
index 88fe75804e584e7fac60f0b89c749cbf57eba72d..b349a406e5e69886dada6c238987292c90af2751 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d
+++ b/sql/hive/src/test/resources/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d
@@ -31,4 +31,4 @@
 24	val_24	2010-03-03	33
 26	val_26	2010-03-03	33
 28	val_28	2010-03-03	33
-37	val_37	2010-03-03	33
\ No newline at end of file
+37	val_37	2010-03-03	33
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e b/sql/hive/src/test/resources/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e
index 170e3b095c5a955013c4f7729fda0d3032a8788b..0a50eb00d58d73a2550588010f9b5112eeb386da 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e
+++ b/sql/hive/src/test/resources/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e
@@ -1,9 +1,9 @@
-key                 	string              	None                
-value               	string              	None                
+key                 	string              	                    
+value               	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-value               	string              	None                
+value               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part14, dbName:default, owner:marmbrus, createTime:1389739459, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1999157106458385464/nzhang_part14, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:value, type:string, comment:null)], parameters:{transient_lastDdlTime=1389739459}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:nzhang_part14, dbName:default, owner:marmbrus, createTime:1413887453, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part14, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:value, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887453}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7
+++ b/sql/hive/src/test/resources/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe
+++ b/sql/hive/src/test/resources/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 b/sql/hive/src/test/resources/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6
index 538ed22d299763d5b78f00a60438a27400796cf6..18b3cb0d99a4ade336c78dca373acb1ce5afd099 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6
+++ b/sql/hive/src/test/resources/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6
@@ -1,2 +1,2 @@
 value= 
-value=__HIVE_DEFAULT_PARTITION__
\ No newline at end of file
+value=__HIVE_DEFAULT_PARTITION__
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e b/sql/hive/src/test/resources/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e
index 9e1bc82fe15f948b6528490e89767ac14008d704..bdbb3fb9b611f19630c86eddf0d86ecb2c998674 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e
+++ b/sql/hive/src/test/resources/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e
@@ -3,4 +3,4 @@ k1	__HIVE_DEFAULT_PARTITION__
 k2	__HIVE_DEFAULT_PARTITION__
 k2	__HIVE_DEFAULT_PARTITION__
 k3	 
-k3	 
\ No newline at end of file
+k3	 
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e b/sql/hive/src/test/resources/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e
index df0090edb6b9e584418e0b7ef26d69c68c5eb1af..c51eca242343449464c62093dfefcc8416ab44af 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e
+++ b/sql/hive/src/test/resources/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e
@@ -1,9 +1,9 @@
-key                 	string              	None                
-value               	string              	None                
+key                 	string              	                    
+value               	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-value               	string              	None                
+value               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part14, dbName:default, owner:marmbrus, createTime:1389738860, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2847673635801001933/nzhang_part14, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:value, type:string, comment:null)], parameters:{transient_lastDdlTime=1389738860}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:nzhang_part14, dbName:default, owner:marmbrus, createTime:1413887510, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part14, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:value, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887510}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7
+++ b/sql/hive/src/test/resources/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe
+++ b/sql/hive/src/test/resources/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 b/sql/hive/src/test/resources/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6
index 538ed22d299763d5b78f00a60438a27400796cf6..18b3cb0d99a4ade336c78dca373acb1ce5afd099 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6
+++ b/sql/hive/src/test/resources/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6
@@ -1,2 +1,2 @@
 value= 
-value=__HIVE_DEFAULT_PARTITION__
\ No newline at end of file
+value=__HIVE_DEFAULT_PARTITION__
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e b/sql/hive/src/test/resources/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e
index 9e1bc82fe15f948b6528490e89767ac14008d704..bdbb3fb9b611f19630c86eddf0d86ecb2c998674 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e
+++ b/sql/hive/src/test/resources/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e
@@ -3,4 +3,4 @@ k1	__HIVE_DEFAULT_PARTITION__
 k2	__HIVE_DEFAULT_PARTITION__
 k2	__HIVE_DEFAULT_PARTITION__
 k3	 
-k3	 
\ No newline at end of file
+k3	 
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c b/sql/hive/src/test/resources/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 b/sql/hive/src/test/resources/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a b/sql/hive/src/test/resources/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 b/sql/hive/src/test/resources/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9
deleted file mode 100644
index 99a66d603300d8bd7da527b230ecd00fba330ac8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9
+++ /dev/null
@@ -1,3 +0,0 @@
-part_key=%7B2
-part_key=1
-part_key=3%5D
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424 b/sql/hive/src/test/resources/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424
index def850839a47b363b2dd015e4110d1f75156e637..5bdf30a775a6ef8c234d4d40249373b89caf213b 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424
+++ b/sql/hive/src/test/resources/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424
@@ -1,12 +1,12 @@
-key                 	string              	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
+key                 	string              	                    
+value               	string              	                    
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-hr                  	string              	None                
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part_bucket, dbName:default, owner:marmbrus, createTime:1389739342, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/nzhang_part_bucket, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:10, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[key], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1389739342}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:nzhang_part_bucket, dbName:default, owner:marmbrus, createTime:1413887567, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part_bucket, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:10, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[key], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887567}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e
+++ b/sql/hive/src/test/resources/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7
+++ b/sql/hive/src/test/resources/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33 b/sql/hive/src/test/resources/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33
index 1f345dad614ad37d75b36296e0fbea2ec94235f8..7342c15206a35acf2af27de4920a1b830c3f6b2e 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33
+++ b/sql/hive/src/test/resources/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33
@@ -1,2 +1,2 @@
 ds=2010-03-23/hr=11
-ds=2010-03-23/hr=12
\ No newline at end of file
+ds=2010-03-23/hr=12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf b/sql/hive/src/test/resources/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf
index 5f936fa91d2f5207f637122dcbe5fa37305b057b..ceb6e5f13d3c5b8770e74921215ce0ee49cb3c03 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf
+++ b/sql/hive/src/test/resources/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf
@@ -997,4 +997,4 @@
 98	val_98	2010-03-23	11
 98	val_98	2010-03-23	11
 98	val_98	2010-03-23	11
-98	val_98	2010-03-23	11
\ No newline at end of file
+98	val_98	2010-03-23	11
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132 b/sql/hive/src/test/resources/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132
index 45c45d0082ee39ec669b32753c422124c730a9cb..bae05fd209e81e1f7b9a1177b6e03fab6b951265 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132
+++ b/sql/hive/src/test/resources/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132
@@ -997,4 +997,4 @@
 98	val_98	2010-03-23	12
 98	val_98	2010-03-23	12
 98	val_98	2010-03-23	12
-98	val_98	2010-03-23	12
\ No newline at end of file
+98	val_98	2010-03-23	12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397 b/sql/hive/src/test/resources/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397
index d35fbec80c19e571b8f5142daeff93c7c6f89b10..8dbbd4321c3675ed297e8c26582f9a74fdf95f06 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397
+++ b/sql/hive/src/test/resources/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397
@@ -1,12 +1,12 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-hr                  	string              	None                
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part3, dbName:default, owner:marmbrus, createTime:1390899609, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part3, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{p3=v3, transient_lastDdlTime=1390899609}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:nzhang_part3, dbName:default, owner:marmbrus, createTime:1413887598, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part3, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887598}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part3-8-c32770da4784bfea4d0dd77fdcba4f0a b/sql/hive/src/test/resources/golden/load_dyn_part3-8-c32770da4784bfea4d0dd77fdcba4f0a
new file mode 100644
index 0000000000000000000000000000000000000000..c38f4300e4058c1f2f7a5f9deff16a1becf4ca86
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/load_dyn_part3-8-c32770da4784bfea4d0dd77fdcba4f0a
@@ -0,0 +1,2000 @@
+0	val_0	2008-04-08	11
+0	val_0	2008-04-08	11
+0	val_0	2008-04-08	11
+2	val_2	2008-04-08	11
+4	val_4	2008-04-08	11
+5	val_5	2008-04-08	11
+5	val_5	2008-04-08	11
+5	val_5	2008-04-08	11
+8	val_8	2008-04-08	11
+9	val_9	2008-04-08	11
+10	val_10	2008-04-08	11
+11	val_11	2008-04-08	11
+12	val_12	2008-04-08	11
+12	val_12	2008-04-08	11
+15	val_15	2008-04-08	11
+15	val_15	2008-04-08	11
+17	val_17	2008-04-08	11
+18	val_18	2008-04-08	11
+18	val_18	2008-04-08	11
+19	val_19	2008-04-08	11
+20	val_20	2008-04-08	11
+24	val_24	2008-04-08	11
+24	val_24	2008-04-08	11
+26	val_26	2008-04-08	11
+26	val_26	2008-04-08	11
+27	val_27	2008-04-08	11
+28	val_28	2008-04-08	11
+30	val_30	2008-04-08	11
+33	val_33	2008-04-08	11
+34	val_34	2008-04-08	11
+35	val_35	2008-04-08	11
+35	val_35	2008-04-08	11
+35	val_35	2008-04-08	11
+37	val_37	2008-04-08	11
+37	val_37	2008-04-08	11
+41	val_41	2008-04-08	11
+42	val_42	2008-04-08	11
+42	val_42	2008-04-08	11
+43	val_43	2008-04-08	11
+44	val_44	2008-04-08	11
+47	val_47	2008-04-08	11
+51	val_51	2008-04-08	11
+51	val_51	2008-04-08	11
+53	val_53	2008-04-08	11
+54	val_54	2008-04-08	11
+57	val_57	2008-04-08	11
+58	val_58	2008-04-08	11
+58	val_58	2008-04-08	11
+64	val_64	2008-04-08	11
+65	val_65	2008-04-08	11
+66	val_66	2008-04-08	11
+67	val_67	2008-04-08	11
+67	val_67	2008-04-08	11
+69	val_69	2008-04-08	11
+70	val_70	2008-04-08	11
+70	val_70	2008-04-08	11
+70	val_70	2008-04-08	11
+72	val_72	2008-04-08	11
+72	val_72	2008-04-08	11
+74	val_74	2008-04-08	11
+76	val_76	2008-04-08	11
+76	val_76	2008-04-08	11
+77	val_77	2008-04-08	11
+78	val_78	2008-04-08	11
+80	val_80	2008-04-08	11
+82	val_82	2008-04-08	11
+83	val_83	2008-04-08	11
+83	val_83	2008-04-08	11
+84	val_84	2008-04-08	11
+84	val_84	2008-04-08	11
+85	val_85	2008-04-08	11
+86	val_86	2008-04-08	11
+87	val_87	2008-04-08	11
+90	val_90	2008-04-08	11
+90	val_90	2008-04-08	11
+90	val_90	2008-04-08	11
+92	val_92	2008-04-08	11
+95	val_95	2008-04-08	11
+95	val_95	2008-04-08	11
+96	val_96	2008-04-08	11
+97	val_97	2008-04-08	11
+97	val_97	2008-04-08	11
+98	val_98	2008-04-08	11
+98	val_98	2008-04-08	11
+100	val_100	2008-04-08	11
+100	val_100	2008-04-08	11
+103	val_103	2008-04-08	11
+103	val_103	2008-04-08	11
+104	val_104	2008-04-08	11
+104	val_104	2008-04-08	11
+105	val_105	2008-04-08	11
+111	val_111	2008-04-08	11
+113	val_113	2008-04-08	11
+113	val_113	2008-04-08	11
+114	val_114	2008-04-08	11
+116	val_116	2008-04-08	11
+118	val_118	2008-04-08	11
+118	val_118	2008-04-08	11
+119	val_119	2008-04-08	11
+119	val_119	2008-04-08	11
+119	val_119	2008-04-08	11
+120	val_120	2008-04-08	11
+120	val_120	2008-04-08	11
+125	val_125	2008-04-08	11
+125	val_125	2008-04-08	11
+126	val_126	2008-04-08	11
+128	val_128	2008-04-08	11
+128	val_128	2008-04-08	11
+128	val_128	2008-04-08	11
+129	val_129	2008-04-08	11
+129	val_129	2008-04-08	11
+131	val_131	2008-04-08	11
+133	val_133	2008-04-08	11
+134	val_134	2008-04-08	11
+134	val_134	2008-04-08	11
+136	val_136	2008-04-08	11
+137	val_137	2008-04-08	11
+137	val_137	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+143	val_143	2008-04-08	11
+145	val_145	2008-04-08	11
+146	val_146	2008-04-08	11
+146	val_146	2008-04-08	11
+149	val_149	2008-04-08	11
+149	val_149	2008-04-08	11
+150	val_150	2008-04-08	11
+152	val_152	2008-04-08	11
+152	val_152	2008-04-08	11
+153	val_153	2008-04-08	11
+155	val_155	2008-04-08	11
+156	val_156	2008-04-08	11
+157	val_157	2008-04-08	11
+158	val_158	2008-04-08	11
+160	val_160	2008-04-08	11
+162	val_162	2008-04-08	11
+163	val_163	2008-04-08	11
+164	val_164	2008-04-08	11
+164	val_164	2008-04-08	11
+165	val_165	2008-04-08	11
+165	val_165	2008-04-08	11
+166	val_166	2008-04-08	11
+167	val_167	2008-04-08	11
+167	val_167	2008-04-08	11
+167	val_167	2008-04-08	11
+168	val_168	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+170	val_170	2008-04-08	11
+172	val_172	2008-04-08	11
+172	val_172	2008-04-08	11
+174	val_174	2008-04-08	11
+174	val_174	2008-04-08	11
+175	val_175	2008-04-08	11
+175	val_175	2008-04-08	11
+176	val_176	2008-04-08	11
+176	val_176	2008-04-08	11
+177	val_177	2008-04-08	11
+178	val_178	2008-04-08	11
+179	val_179	2008-04-08	11
+179	val_179	2008-04-08	11
+180	val_180	2008-04-08	11
+181	val_181	2008-04-08	11
+183	val_183	2008-04-08	11
+186	val_186	2008-04-08	11
+187	val_187	2008-04-08	11
+187	val_187	2008-04-08	11
+187	val_187	2008-04-08	11
+189	val_189	2008-04-08	11
+190	val_190	2008-04-08	11
+191	val_191	2008-04-08	11
+191	val_191	2008-04-08	11
+192	val_192	2008-04-08	11
+193	val_193	2008-04-08	11
+193	val_193	2008-04-08	11
+193	val_193	2008-04-08	11
+194	val_194	2008-04-08	11
+195	val_195	2008-04-08	11
+195	val_195	2008-04-08	11
+196	val_196	2008-04-08	11
+197	val_197	2008-04-08	11
+197	val_197	2008-04-08	11
+199	val_199	2008-04-08	11
+199	val_199	2008-04-08	11
+199	val_199	2008-04-08	11
+200	val_200	2008-04-08	11
+200	val_200	2008-04-08	11
+201	val_201	2008-04-08	11
+202	val_202	2008-04-08	11
+203	val_203	2008-04-08	11
+203	val_203	2008-04-08	11
+205	val_205	2008-04-08	11
+205	val_205	2008-04-08	11
+207	val_207	2008-04-08	11
+207	val_207	2008-04-08	11
+208	val_208	2008-04-08	11
+208	val_208	2008-04-08	11
+208	val_208	2008-04-08	11
+209	val_209	2008-04-08	11
+209	val_209	2008-04-08	11
+213	val_213	2008-04-08	11
+213	val_213	2008-04-08	11
+214	val_214	2008-04-08	11
+216	val_216	2008-04-08	11
+216	val_216	2008-04-08	11
+217	val_217	2008-04-08	11
+217	val_217	2008-04-08	11
+218	val_218	2008-04-08	11
+219	val_219	2008-04-08	11
+219	val_219	2008-04-08	11
+221	val_221	2008-04-08	11
+221	val_221	2008-04-08	11
+222	val_222	2008-04-08	11
+223	val_223	2008-04-08	11
+223	val_223	2008-04-08	11
+224	val_224	2008-04-08	11
+224	val_224	2008-04-08	11
+226	val_226	2008-04-08	11
+228	val_228	2008-04-08	11
+229	val_229	2008-04-08	11
+229	val_229	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+233	val_233	2008-04-08	11
+233	val_233	2008-04-08	11
+235	val_235	2008-04-08	11
+237	val_237	2008-04-08	11
+237	val_237	2008-04-08	11
+238	val_238	2008-04-08	11
+238	val_238	2008-04-08	11
+239	val_239	2008-04-08	11
+239	val_239	2008-04-08	11
+241	val_241	2008-04-08	11
+242	val_242	2008-04-08	11
+242	val_242	2008-04-08	11
+244	val_244	2008-04-08	11
+247	val_247	2008-04-08	11
+248	val_248	2008-04-08	11
+249	val_249	2008-04-08	11
+252	val_252	2008-04-08	11
+255	val_255	2008-04-08	11
+255	val_255	2008-04-08	11
+256	val_256	2008-04-08	11
+256	val_256	2008-04-08	11
+257	val_257	2008-04-08	11
+258	val_258	2008-04-08	11
+260	val_260	2008-04-08	11
+262	val_262	2008-04-08	11
+263	val_263	2008-04-08	11
+265	val_265	2008-04-08	11
+265	val_265	2008-04-08	11
+266	val_266	2008-04-08	11
+272	val_272	2008-04-08	11
+272	val_272	2008-04-08	11
+273	val_273	2008-04-08	11
+273	val_273	2008-04-08	11
+273	val_273	2008-04-08	11
+274	val_274	2008-04-08	11
+275	val_275	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+278	val_278	2008-04-08	11
+278	val_278	2008-04-08	11
+280	val_280	2008-04-08	11
+280	val_280	2008-04-08	11
+281	val_281	2008-04-08	11
+281	val_281	2008-04-08	11
+282	val_282	2008-04-08	11
+282	val_282	2008-04-08	11
+283	val_283	2008-04-08	11
+284	val_284	2008-04-08	11
+285	val_285	2008-04-08	11
+286	val_286	2008-04-08	11
+287	val_287	2008-04-08	11
+288	val_288	2008-04-08	11
+288	val_288	2008-04-08	11
+289	val_289	2008-04-08	11
+291	val_291	2008-04-08	11
+292	val_292	2008-04-08	11
+296	val_296	2008-04-08	11
+298	val_298	2008-04-08	11
+298	val_298	2008-04-08	11
+298	val_298	2008-04-08	11
+302	val_302	2008-04-08	11
+305	val_305	2008-04-08	11
+306	val_306	2008-04-08	11
+307	val_307	2008-04-08	11
+307	val_307	2008-04-08	11
+308	val_308	2008-04-08	11
+309	val_309	2008-04-08	11
+309	val_309	2008-04-08	11
+310	val_310	2008-04-08	11
+311	val_311	2008-04-08	11
+311	val_311	2008-04-08	11
+311	val_311	2008-04-08	11
+315	val_315	2008-04-08	11
+316	val_316	2008-04-08	11
+316	val_316	2008-04-08	11
+316	val_316	2008-04-08	11
+317	val_317	2008-04-08	11
+317	val_317	2008-04-08	11
+318	val_318	2008-04-08	11
+318	val_318	2008-04-08	11
+318	val_318	2008-04-08	11
+321	val_321	2008-04-08	11
+321	val_321	2008-04-08	11
+322	val_322	2008-04-08	11
+322	val_322	2008-04-08	11
+323	val_323	2008-04-08	11
+325	val_325	2008-04-08	11
+325	val_325	2008-04-08	11
+327	val_327	2008-04-08	11
+327	val_327	2008-04-08	11
+327	val_327	2008-04-08	11
+331	val_331	2008-04-08	11
+331	val_331	2008-04-08	11
+332	val_332	2008-04-08	11
+333	val_333	2008-04-08	11
+333	val_333	2008-04-08	11
+335	val_335	2008-04-08	11
+336	val_336	2008-04-08	11
+338	val_338	2008-04-08	11
+339	val_339	2008-04-08	11
+341	val_341	2008-04-08	11
+342	val_342	2008-04-08	11
+342	val_342	2008-04-08	11
+344	val_344	2008-04-08	11
+344	val_344	2008-04-08	11
+345	val_345	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+351	val_351	2008-04-08	11
+353	val_353	2008-04-08	11
+353	val_353	2008-04-08	11
+356	val_356	2008-04-08	11
+360	val_360	2008-04-08	11
+362	val_362	2008-04-08	11
+364	val_364	2008-04-08	11
+365	val_365	2008-04-08	11
+366	val_366	2008-04-08	11
+367	val_367	2008-04-08	11
+367	val_367	2008-04-08	11
+368	val_368	2008-04-08	11
+369	val_369	2008-04-08	11
+369	val_369	2008-04-08	11
+369	val_369	2008-04-08	11
+373	val_373	2008-04-08	11
+374	val_374	2008-04-08	11
+375	val_375	2008-04-08	11
+377	val_377	2008-04-08	11
+378	val_378	2008-04-08	11
+379	val_379	2008-04-08	11
+382	val_382	2008-04-08	11
+382	val_382	2008-04-08	11
+384	val_384	2008-04-08	11
+384	val_384	2008-04-08	11
+384	val_384	2008-04-08	11
+386	val_386	2008-04-08	11
+389	val_389	2008-04-08	11
+392	val_392	2008-04-08	11
+393	val_393	2008-04-08	11
+394	val_394	2008-04-08	11
+395	val_395	2008-04-08	11
+395	val_395	2008-04-08	11
+396	val_396	2008-04-08	11
+396	val_396	2008-04-08	11
+396	val_396	2008-04-08	11
+397	val_397	2008-04-08	11
+397	val_397	2008-04-08	11
+399	val_399	2008-04-08	11
+399	val_399	2008-04-08	11
+400	val_400	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+402	val_402	2008-04-08	11
+403	val_403	2008-04-08	11
+403	val_403	2008-04-08	11
+403	val_403	2008-04-08	11
+404	val_404	2008-04-08	11
+404	val_404	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+407	val_407	2008-04-08	11
+409	val_409	2008-04-08	11
+409	val_409	2008-04-08	11
+409	val_409	2008-04-08	11
+411	val_411	2008-04-08	11
+413	val_413	2008-04-08	11
+413	val_413	2008-04-08	11
+414	val_414	2008-04-08	11
+414	val_414	2008-04-08	11
+417	val_417	2008-04-08	11
+417	val_417	2008-04-08	11
+417	val_417	2008-04-08	11
+418	val_418	2008-04-08	11
+419	val_419	2008-04-08	11
+421	val_421	2008-04-08	11
+424	val_424	2008-04-08	11
+424	val_424	2008-04-08	11
+427	val_427	2008-04-08	11
+429	val_429	2008-04-08	11
+429	val_429	2008-04-08	11
+430	val_430	2008-04-08	11
+430	val_430	2008-04-08	11
+430	val_430	2008-04-08	11
+431	val_431	2008-04-08	11
+431	val_431	2008-04-08	11
+431	val_431	2008-04-08	11
+432	val_432	2008-04-08	11
+435	val_435	2008-04-08	11
+436	val_436	2008-04-08	11
+437	val_437	2008-04-08	11
+438	val_438	2008-04-08	11
+438	val_438	2008-04-08	11
+438	val_438	2008-04-08	11
+439	val_439	2008-04-08	11
+439	val_439	2008-04-08	11
+443	val_443	2008-04-08	11
+444	val_444	2008-04-08	11
+446	val_446	2008-04-08	11
+448	val_448	2008-04-08	11
+449	val_449	2008-04-08	11
+452	val_452	2008-04-08	11
+453	val_453	2008-04-08	11
+454	val_454	2008-04-08	11
+454	val_454	2008-04-08	11
+454	val_454	2008-04-08	11
+455	val_455	2008-04-08	11
+457	val_457	2008-04-08	11
+458	val_458	2008-04-08	11
+458	val_458	2008-04-08	11
+459	val_459	2008-04-08	11
+459	val_459	2008-04-08	11
+460	val_460	2008-04-08	11
+462	val_462	2008-04-08	11
+462	val_462	2008-04-08	11
+463	val_463	2008-04-08	11
+463	val_463	2008-04-08	11
+466	val_466	2008-04-08	11
+466	val_466	2008-04-08	11
+466	val_466	2008-04-08	11
+467	val_467	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+470	val_470	2008-04-08	11
+472	val_472	2008-04-08	11
+475	val_475	2008-04-08	11
+477	val_477	2008-04-08	11
+478	val_478	2008-04-08	11
+478	val_478	2008-04-08	11
+479	val_479	2008-04-08	11
+480	val_480	2008-04-08	11
+480	val_480	2008-04-08	11
+480	val_480	2008-04-08	11
+481	val_481	2008-04-08	11
+482	val_482	2008-04-08	11
+483	val_483	2008-04-08	11
+484	val_484	2008-04-08	11
+485	val_485	2008-04-08	11
+487	val_487	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+490	val_490	2008-04-08	11
+491	val_491	2008-04-08	11
+492	val_492	2008-04-08	11
+492	val_492	2008-04-08	11
+493	val_493	2008-04-08	11
+494	val_494	2008-04-08	11
+495	val_495	2008-04-08	11
+496	val_496	2008-04-08	11
+497	val_497	2008-04-08	11
+498	val_498	2008-04-08	11
+498	val_498	2008-04-08	11
+498	val_498	2008-04-08	11
+0	val_0	2008-04-08	12
+0	val_0	2008-04-08	12
+0	val_0	2008-04-08	12
+2	val_2	2008-04-08	12
+4	val_4	2008-04-08	12
+5	val_5	2008-04-08	12
+5	val_5	2008-04-08	12
+5	val_5	2008-04-08	12
+8	val_8	2008-04-08	12
+9	val_9	2008-04-08	12
+10	val_10	2008-04-08	12
+11	val_11	2008-04-08	12
+12	val_12	2008-04-08	12
+12	val_12	2008-04-08	12
+15	val_15	2008-04-08	12
+15	val_15	2008-04-08	12
+17	val_17	2008-04-08	12
+18	val_18	2008-04-08	12
+18	val_18	2008-04-08	12
+19	val_19	2008-04-08	12
+20	val_20	2008-04-08	12
+24	val_24	2008-04-08	12
+24	val_24	2008-04-08	12
+26	val_26	2008-04-08	12
+26	val_26	2008-04-08	12
+27	val_27	2008-04-08	12
+28	val_28	2008-04-08	12
+30	val_30	2008-04-08	12
+33	val_33	2008-04-08	12
+34	val_34	2008-04-08	12
+35	val_35	2008-04-08	12
+35	val_35	2008-04-08	12
+35	val_35	2008-04-08	12
+37	val_37	2008-04-08	12
+37	val_37	2008-04-08	12
+41	val_41	2008-04-08	12
+42	val_42	2008-04-08	12
+42	val_42	2008-04-08	12
+43	val_43	2008-04-08	12
+44	val_44	2008-04-08	12
+47	val_47	2008-04-08	12
+51	val_51	2008-04-08	12
+51	val_51	2008-04-08	12
+53	val_53	2008-04-08	12
+54	val_54	2008-04-08	12
+57	val_57	2008-04-08	12
+58	val_58	2008-04-08	12
+58	val_58	2008-04-08	12
+64	val_64	2008-04-08	12
+65	val_65	2008-04-08	12
+66	val_66	2008-04-08	12
+67	val_67	2008-04-08	12
+67	val_67	2008-04-08	12
+69	val_69	2008-04-08	12
+70	val_70	2008-04-08	12
+70	val_70	2008-04-08	12
+70	val_70	2008-04-08	12
+72	val_72	2008-04-08	12
+72	val_72	2008-04-08	12
+74	val_74	2008-04-08	12
+76	val_76	2008-04-08	12
+76	val_76	2008-04-08	12
+77	val_77	2008-04-08	12
+78	val_78	2008-04-08	12
+80	val_80	2008-04-08	12
+82	val_82	2008-04-08	12
+83	val_83	2008-04-08	12
+83	val_83	2008-04-08	12
+84	val_84	2008-04-08	12
+84	val_84	2008-04-08	12
+85	val_85	2008-04-08	12
+86	val_86	2008-04-08	12
+87	val_87	2008-04-08	12
+90	val_90	2008-04-08	12
+90	val_90	2008-04-08	12
+90	val_90	2008-04-08	12
+92	val_92	2008-04-08	12
+95	val_95	2008-04-08	12
+95	val_95	2008-04-08	12
+96	val_96	2008-04-08	12
+97	val_97	2008-04-08	12
+97	val_97	2008-04-08	12
+98	val_98	2008-04-08	12
+98	val_98	2008-04-08	12
+100	val_100	2008-04-08	12
+100	val_100	2008-04-08	12
+103	val_103	2008-04-08	12
+103	val_103	2008-04-08	12
+104	val_104	2008-04-08	12
+104	val_104	2008-04-08	12
+105	val_105	2008-04-08	12
+111	val_111	2008-04-08	12
+113	val_113	2008-04-08	12
+113	val_113	2008-04-08	12
+114	val_114	2008-04-08	12
+116	val_116	2008-04-08	12
+118	val_118	2008-04-08	12
+118	val_118	2008-04-08	12
+119	val_119	2008-04-08	12
+119	val_119	2008-04-08	12
+119	val_119	2008-04-08	12
+120	val_120	2008-04-08	12
+120	val_120	2008-04-08	12
+125	val_125	2008-04-08	12
+125	val_125	2008-04-08	12
+126	val_126	2008-04-08	12
+128	val_128	2008-04-08	12
+128	val_128	2008-04-08	12
+128	val_128	2008-04-08	12
+129	val_129	2008-04-08	12
+129	val_129	2008-04-08	12
+131	val_131	2008-04-08	12
+133	val_133	2008-04-08	12
+134	val_134	2008-04-08	12
+134	val_134	2008-04-08	12
+136	val_136	2008-04-08	12
+137	val_137	2008-04-08	12
+137	val_137	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+143	val_143	2008-04-08	12
+145	val_145	2008-04-08	12
+146	val_146	2008-04-08	12
+146	val_146	2008-04-08	12
+149	val_149	2008-04-08	12
+149	val_149	2008-04-08	12
+150	val_150	2008-04-08	12
+152	val_152	2008-04-08	12
+152	val_152	2008-04-08	12
+153	val_153	2008-04-08	12
+155	val_155	2008-04-08	12
+156	val_156	2008-04-08	12
+157	val_157	2008-04-08	12
+158	val_158	2008-04-08	12
+160	val_160	2008-04-08	12
+162	val_162	2008-04-08	12
+163	val_163	2008-04-08	12
+164	val_164	2008-04-08	12
+164	val_164	2008-04-08	12
+165	val_165	2008-04-08	12
+165	val_165	2008-04-08	12
+166	val_166	2008-04-08	12
+167	val_167	2008-04-08	12
+167	val_167	2008-04-08	12
+167	val_167	2008-04-08	12
+168	val_168	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+170	val_170	2008-04-08	12
+172	val_172	2008-04-08	12
+172	val_172	2008-04-08	12
+174	val_174	2008-04-08	12
+174	val_174	2008-04-08	12
+175	val_175	2008-04-08	12
+175	val_175	2008-04-08	12
+176	val_176	2008-04-08	12
+176	val_176	2008-04-08	12
+177	val_177	2008-04-08	12
+178	val_178	2008-04-08	12
+179	val_179	2008-04-08	12
+179	val_179	2008-04-08	12
+180	val_180	2008-04-08	12
+181	val_181	2008-04-08	12
+183	val_183	2008-04-08	12
+186	val_186	2008-04-08	12
+187	val_187	2008-04-08	12
+187	val_187	2008-04-08	12
+187	val_187	2008-04-08	12
+189	val_189	2008-04-08	12
+190	val_190	2008-04-08	12
+191	val_191	2008-04-08	12
+191	val_191	2008-04-08	12
+192	val_192	2008-04-08	12
+193	val_193	2008-04-08	12
+193	val_193	2008-04-08	12
+193	val_193	2008-04-08	12
+194	val_194	2008-04-08	12
+195	val_195	2008-04-08	12
+195	val_195	2008-04-08	12
+196	val_196	2008-04-08	12
+197	val_197	2008-04-08	12
+197	val_197	2008-04-08	12
+199	val_199	2008-04-08	12
+199	val_199	2008-04-08	12
+199	val_199	2008-04-08	12
+200	val_200	2008-04-08	12
+200	val_200	2008-04-08	12
+201	val_201	2008-04-08	12
+202	val_202	2008-04-08	12
+203	val_203	2008-04-08	12
+203	val_203	2008-04-08	12
+205	val_205	2008-04-08	12
+205	val_205	2008-04-08	12
+207	val_207	2008-04-08	12
+207	val_207	2008-04-08	12
+208	val_208	2008-04-08	12
+208	val_208	2008-04-08	12
+208	val_208	2008-04-08	12
+209	val_209	2008-04-08	12
+209	val_209	2008-04-08	12
+213	val_213	2008-04-08	12
+213	val_213	2008-04-08	12
+214	val_214	2008-04-08	12
+216	val_216	2008-04-08	12
+216	val_216	2008-04-08	12
+217	val_217	2008-04-08	12
+217	val_217	2008-04-08	12
+218	val_218	2008-04-08	12
+219	val_219	2008-04-08	12
+219	val_219	2008-04-08	12
+221	val_221	2008-04-08	12
+221	val_221	2008-04-08	12
+222	val_222	2008-04-08	12
+223	val_223	2008-04-08	12
+223	val_223	2008-04-08	12
+224	val_224	2008-04-08	12
+224	val_224	2008-04-08	12
+226	val_226	2008-04-08	12
+228	val_228	2008-04-08	12
+229	val_229	2008-04-08	12
+229	val_229	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+233	val_233	2008-04-08	12
+233	val_233	2008-04-08	12
+235	val_235	2008-04-08	12
+237	val_237	2008-04-08	12
+237	val_237	2008-04-08	12
+238	val_238	2008-04-08	12
+238	val_238	2008-04-08	12
+239	val_239	2008-04-08	12
+239	val_239	2008-04-08	12
+241	val_241	2008-04-08	12
+242	val_242	2008-04-08	12
+242	val_242	2008-04-08	12
+244	val_244	2008-04-08	12
+247	val_247	2008-04-08	12
+248	val_248	2008-04-08	12
+249	val_249	2008-04-08	12
+252	val_252	2008-04-08	12
+255	val_255	2008-04-08	12
+255	val_255	2008-04-08	12
+256	val_256	2008-04-08	12
+256	val_256	2008-04-08	12
+257	val_257	2008-04-08	12
+258	val_258	2008-04-08	12
+260	val_260	2008-04-08	12
+262	val_262	2008-04-08	12
+263	val_263	2008-04-08	12
+265	val_265	2008-04-08	12
+265	val_265	2008-04-08	12
+266	val_266	2008-04-08	12
+272	val_272	2008-04-08	12
+272	val_272	2008-04-08	12
+273	val_273	2008-04-08	12
+273	val_273	2008-04-08	12
+273	val_273	2008-04-08	12
+274	val_274	2008-04-08	12
+275	val_275	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+278	val_278	2008-04-08	12
+278	val_278	2008-04-08	12
+280	val_280	2008-04-08	12
+280	val_280	2008-04-08	12
+281	val_281	2008-04-08	12
+281	val_281	2008-04-08	12
+282	val_282	2008-04-08	12
+282	val_282	2008-04-08	12
+283	val_283	2008-04-08	12
+284	val_284	2008-04-08	12
+285	val_285	2008-04-08	12
+286	val_286	2008-04-08	12
+287	val_287	2008-04-08	12
+288	val_288	2008-04-08	12
+288	val_288	2008-04-08	12
+289	val_289	2008-04-08	12
+291	val_291	2008-04-08	12
+292	val_292	2008-04-08	12
+296	val_296	2008-04-08	12
+298	val_298	2008-04-08	12
+298	val_298	2008-04-08	12
+298	val_298	2008-04-08	12
+302	val_302	2008-04-08	12
+305	val_305	2008-04-08	12
+306	val_306	2008-04-08	12
+307	val_307	2008-04-08	12
+307	val_307	2008-04-08	12
+308	val_308	2008-04-08	12
+309	val_309	2008-04-08	12
+309	val_309	2008-04-08	12
+310	val_310	2008-04-08	12
+311	val_311	2008-04-08	12
+311	val_311	2008-04-08	12
+311	val_311	2008-04-08	12
+315	val_315	2008-04-08	12
+316	val_316	2008-04-08	12
+316	val_316	2008-04-08	12
+316	val_316	2008-04-08	12
+317	val_317	2008-04-08	12
+317	val_317	2008-04-08	12
+318	val_318	2008-04-08	12
+318	val_318	2008-04-08	12
+318	val_318	2008-04-08	12
+321	val_321	2008-04-08	12
+321	val_321	2008-04-08	12
+322	val_322	2008-04-08	12
+322	val_322	2008-04-08	12
+323	val_323	2008-04-08	12
+325	val_325	2008-04-08	12
+325	val_325	2008-04-08	12
+327	val_327	2008-04-08	12
+327	val_327	2008-04-08	12
+327	val_327	2008-04-08	12
+331	val_331	2008-04-08	12
+331	val_331	2008-04-08	12
+332	val_332	2008-04-08	12
+333	val_333	2008-04-08	12
+333	val_333	2008-04-08	12
+335	val_335	2008-04-08	12
+336	val_336	2008-04-08	12
+338	val_338	2008-04-08	12
+339	val_339	2008-04-08	12
+341	val_341	2008-04-08	12
+342	val_342	2008-04-08	12
+342	val_342	2008-04-08	12
+344	val_344	2008-04-08	12
+344	val_344	2008-04-08	12
+345	val_345	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+351	val_351	2008-04-08	12
+353	val_353	2008-04-08	12
+353	val_353	2008-04-08	12
+356	val_356	2008-04-08	12
+360	val_360	2008-04-08	12
+362	val_362	2008-04-08	12
+364	val_364	2008-04-08	12
+365	val_365	2008-04-08	12
+366	val_366	2008-04-08	12
+367	val_367	2008-04-08	12
+367	val_367	2008-04-08	12
+368	val_368	2008-04-08	12
+369	val_369	2008-04-08	12
+369	val_369	2008-04-08	12
+369	val_369	2008-04-08	12
+373	val_373	2008-04-08	12
+374	val_374	2008-04-08	12
+375	val_375	2008-04-08	12
+377	val_377	2008-04-08	12
+378	val_378	2008-04-08	12
+379	val_379	2008-04-08	12
+382	val_382	2008-04-08	12
+382	val_382	2008-04-08	12
+384	val_384	2008-04-08	12
+384	val_384	2008-04-08	12
+384	val_384	2008-04-08	12
+386	val_386	2008-04-08	12
+389	val_389	2008-04-08	12
+392	val_392	2008-04-08	12
+393	val_393	2008-04-08	12
+394	val_394	2008-04-08	12
+395	val_395	2008-04-08	12
+395	val_395	2008-04-08	12
+396	val_396	2008-04-08	12
+396	val_396	2008-04-08	12
+396	val_396	2008-04-08	12
+397	val_397	2008-04-08	12
+397	val_397	2008-04-08	12
+399	val_399	2008-04-08	12
+399	val_399	2008-04-08	12
+400	val_400	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+402	val_402	2008-04-08	12
+403	val_403	2008-04-08	12
+403	val_403	2008-04-08	12
+403	val_403	2008-04-08	12
+404	val_404	2008-04-08	12
+404	val_404	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+407	val_407	2008-04-08	12
+409	val_409	2008-04-08	12
+409	val_409	2008-04-08	12
+409	val_409	2008-04-08	12
+411	val_411	2008-04-08	12
+413	val_413	2008-04-08	12
+413	val_413	2008-04-08	12
+414	val_414	2008-04-08	12
+414	val_414	2008-04-08	12
+417	val_417	2008-04-08	12
+417	val_417	2008-04-08	12
+417	val_417	2008-04-08	12
+418	val_418	2008-04-08	12
+419	val_419	2008-04-08	12
+421	val_421	2008-04-08	12
+424	val_424	2008-04-08	12
+424	val_424	2008-04-08	12
+427	val_427	2008-04-08	12
+429	val_429	2008-04-08	12
+429	val_429	2008-04-08	12
+430	val_430	2008-04-08	12
+430	val_430	2008-04-08	12
+430	val_430	2008-04-08	12
+431	val_431	2008-04-08	12
+431	val_431	2008-04-08	12
+431	val_431	2008-04-08	12
+432	val_432	2008-04-08	12
+435	val_435	2008-04-08	12
+436	val_436	2008-04-08	12
+437	val_437	2008-04-08	12
+438	val_438	2008-04-08	12
+438	val_438	2008-04-08	12
+438	val_438	2008-04-08	12
+439	val_439	2008-04-08	12
+439	val_439	2008-04-08	12
+443	val_443	2008-04-08	12
+444	val_444	2008-04-08	12
+446	val_446	2008-04-08	12
+448	val_448	2008-04-08	12
+449	val_449	2008-04-08	12
+452	val_452	2008-04-08	12
+453	val_453	2008-04-08	12
+454	val_454	2008-04-08	12
+454	val_454	2008-04-08	12
+454	val_454	2008-04-08	12
+455	val_455	2008-04-08	12
+457	val_457	2008-04-08	12
+458	val_458	2008-04-08	12
+458	val_458	2008-04-08	12
+459	val_459	2008-04-08	12
+459	val_459	2008-04-08	12
+460	val_460	2008-04-08	12
+462	val_462	2008-04-08	12
+462	val_462	2008-04-08	12
+463	val_463	2008-04-08	12
+463	val_463	2008-04-08	12
+466	val_466	2008-04-08	12
+466	val_466	2008-04-08	12
+466	val_466	2008-04-08	12
+467	val_467	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+470	val_470	2008-04-08	12
+472	val_472	2008-04-08	12
+475	val_475	2008-04-08	12
+477	val_477	2008-04-08	12
+478	val_478	2008-04-08	12
+478	val_478	2008-04-08	12
+479	val_479	2008-04-08	12
+480	val_480	2008-04-08	12
+480	val_480	2008-04-08	12
+480	val_480	2008-04-08	12
+481	val_481	2008-04-08	12
+482	val_482	2008-04-08	12
+483	val_483	2008-04-08	12
+484	val_484	2008-04-08	12
+485	val_485	2008-04-08	12
+487	val_487	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+490	val_490	2008-04-08	12
+491	val_491	2008-04-08	12
+492	val_492	2008-04-08	12
+492	val_492	2008-04-08	12
+493	val_493	2008-04-08	12
+494	val_494	2008-04-08	12
+495	val_495	2008-04-08	12
+496	val_496	2008-04-08	12
+497	val_497	2008-04-08	12
+498	val_498	2008-04-08	12
+498	val_498	2008-04-08	12
+498	val_498	2008-04-08	12
+0	val_0	2008-04-09	11
+0	val_0	2008-04-09	11
+0	val_0	2008-04-09	11
+2	val_2	2008-04-09	11
+4	val_4	2008-04-09	11
+5	val_5	2008-04-09	11
+5	val_5	2008-04-09	11
+5	val_5	2008-04-09	11
+8	val_8	2008-04-09	11
+9	val_9	2008-04-09	11
+10	val_10	2008-04-09	11
+11	val_11	2008-04-09	11
+12	val_12	2008-04-09	11
+12	val_12	2008-04-09	11
+15	val_15	2008-04-09	11
+15	val_15	2008-04-09	11
+17	val_17	2008-04-09	11
+18	val_18	2008-04-09	11
+18	val_18	2008-04-09	11
+19	val_19	2008-04-09	11
+20	val_20	2008-04-09	11
+24	val_24	2008-04-09	11
+24	val_24	2008-04-09	11
+26	val_26	2008-04-09	11
+26	val_26	2008-04-09	11
+27	val_27	2008-04-09	11
+28	val_28	2008-04-09	11
+30	val_30	2008-04-09	11
+33	val_33	2008-04-09	11
+34	val_34	2008-04-09	11
+35	val_35	2008-04-09	11
+35	val_35	2008-04-09	11
+35	val_35	2008-04-09	11
+37	val_37	2008-04-09	11
+37	val_37	2008-04-09	11
+41	val_41	2008-04-09	11
+42	val_42	2008-04-09	11
+42	val_42	2008-04-09	11
+43	val_43	2008-04-09	11
+44	val_44	2008-04-09	11
+47	val_47	2008-04-09	11
+51	val_51	2008-04-09	11
+51	val_51	2008-04-09	11
+53	val_53	2008-04-09	11
+54	val_54	2008-04-09	11
+57	val_57	2008-04-09	11
+58	val_58	2008-04-09	11
+58	val_58	2008-04-09	11
+64	val_64	2008-04-09	11
+65	val_65	2008-04-09	11
+66	val_66	2008-04-09	11
+67	val_67	2008-04-09	11
+67	val_67	2008-04-09	11
+69	val_69	2008-04-09	11
+70	val_70	2008-04-09	11
+70	val_70	2008-04-09	11
+70	val_70	2008-04-09	11
+72	val_72	2008-04-09	11
+72	val_72	2008-04-09	11
+74	val_74	2008-04-09	11
+76	val_76	2008-04-09	11
+76	val_76	2008-04-09	11
+77	val_77	2008-04-09	11
+78	val_78	2008-04-09	11
+80	val_80	2008-04-09	11
+82	val_82	2008-04-09	11
+83	val_83	2008-04-09	11
+83	val_83	2008-04-09	11
+84	val_84	2008-04-09	11
+84	val_84	2008-04-09	11
+85	val_85	2008-04-09	11
+86	val_86	2008-04-09	11
+87	val_87	2008-04-09	11
+90	val_90	2008-04-09	11
+90	val_90	2008-04-09	11
+90	val_90	2008-04-09	11
+92	val_92	2008-04-09	11
+95	val_95	2008-04-09	11
+95	val_95	2008-04-09	11
+96	val_96	2008-04-09	11
+97	val_97	2008-04-09	11
+97	val_97	2008-04-09	11
+98	val_98	2008-04-09	11
+98	val_98	2008-04-09	11
+100	val_100	2008-04-09	11
+100	val_100	2008-04-09	11
+103	val_103	2008-04-09	11
+103	val_103	2008-04-09	11
+104	val_104	2008-04-09	11
+104	val_104	2008-04-09	11
+105	val_105	2008-04-09	11
+111	val_111	2008-04-09	11
+113	val_113	2008-04-09	11
+113	val_113	2008-04-09	11
+114	val_114	2008-04-09	11
+116	val_116	2008-04-09	11
+118	val_118	2008-04-09	11
+118	val_118	2008-04-09	11
+119	val_119	2008-04-09	11
+119	val_119	2008-04-09	11
+119	val_119	2008-04-09	11
+120	val_120	2008-04-09	11
+120	val_120	2008-04-09	11
+125	val_125	2008-04-09	11
+125	val_125	2008-04-09	11
+126	val_126	2008-04-09	11
+128	val_128	2008-04-09	11
+128	val_128	2008-04-09	11
+128	val_128	2008-04-09	11
+129	val_129	2008-04-09	11
+129	val_129	2008-04-09	11
+131	val_131	2008-04-09	11
+133	val_133	2008-04-09	11
+134	val_134	2008-04-09	11
+134	val_134	2008-04-09	11
+136	val_136	2008-04-09	11
+137	val_137	2008-04-09	11
+137	val_137	2008-04-09	11
+138	val_138	2008-04-09	11
+138	val_138	2008-04-09	11
+138	val_138	2008-04-09	11
+138	val_138	2008-04-09	11
+143	val_143	2008-04-09	11
+145	val_145	2008-04-09	11
+146	val_146	2008-04-09	11
+146	val_146	2008-04-09	11
+149	val_149	2008-04-09	11
+149	val_149	2008-04-09	11
+150	val_150	2008-04-09	11
+152	val_152	2008-04-09	11
+152	val_152	2008-04-09	11
+153	val_153	2008-04-09	11
+155	val_155	2008-04-09	11
+156	val_156	2008-04-09	11
+157	val_157	2008-04-09	11
+158	val_158	2008-04-09	11
+160	val_160	2008-04-09	11
+162	val_162	2008-04-09	11
+163	val_163	2008-04-09	11
+164	val_164	2008-04-09	11
+164	val_164	2008-04-09	11
+165	val_165	2008-04-09	11
+165	val_165	2008-04-09	11
+166	val_166	2008-04-09	11
+167	val_167	2008-04-09	11
+167	val_167	2008-04-09	11
+167	val_167	2008-04-09	11
+168	val_168	2008-04-09	11
+169	val_169	2008-04-09	11
+169	val_169	2008-04-09	11
+169	val_169	2008-04-09	11
+169	val_169	2008-04-09	11
+170	val_170	2008-04-09	11
+172	val_172	2008-04-09	11
+172	val_172	2008-04-09	11
+174	val_174	2008-04-09	11
+174	val_174	2008-04-09	11
+175	val_175	2008-04-09	11
+175	val_175	2008-04-09	11
+176	val_176	2008-04-09	11
+176	val_176	2008-04-09	11
+177	val_177	2008-04-09	11
+178	val_178	2008-04-09	11
+179	val_179	2008-04-09	11
+179	val_179	2008-04-09	11
+180	val_180	2008-04-09	11
+181	val_181	2008-04-09	11
+183	val_183	2008-04-09	11
+186	val_186	2008-04-09	11
+187	val_187	2008-04-09	11
+187	val_187	2008-04-09	11
+187	val_187	2008-04-09	11
+189	val_189	2008-04-09	11
+190	val_190	2008-04-09	11
+191	val_191	2008-04-09	11
+191	val_191	2008-04-09	11
+192	val_192	2008-04-09	11
+193	val_193	2008-04-09	11
+193	val_193	2008-04-09	11
+193	val_193	2008-04-09	11
+194	val_194	2008-04-09	11
+195	val_195	2008-04-09	11
+195	val_195	2008-04-09	11
+196	val_196	2008-04-09	11
+197	val_197	2008-04-09	11
+197	val_197	2008-04-09	11
+199	val_199	2008-04-09	11
+199	val_199	2008-04-09	11
+199	val_199	2008-04-09	11
+200	val_200	2008-04-09	11
+200	val_200	2008-04-09	11
+201	val_201	2008-04-09	11
+202	val_202	2008-04-09	11
+203	val_203	2008-04-09	11
+203	val_203	2008-04-09	11
+205	val_205	2008-04-09	11
+205	val_205	2008-04-09	11
+207	val_207	2008-04-09	11
+207	val_207	2008-04-09	11
+208	val_208	2008-04-09	11
+208	val_208	2008-04-09	11
+208	val_208	2008-04-09	11
+209	val_209	2008-04-09	11
+209	val_209	2008-04-09	11
+213	val_213	2008-04-09	11
+213	val_213	2008-04-09	11
+214	val_214	2008-04-09	11
+216	val_216	2008-04-09	11
+216	val_216	2008-04-09	11
+217	val_217	2008-04-09	11
+217	val_217	2008-04-09	11
+218	val_218	2008-04-09	11
+219	val_219	2008-04-09	11
+219	val_219	2008-04-09	11
+221	val_221	2008-04-09	11
+221	val_221	2008-04-09	11
+222	val_222	2008-04-09	11
+223	val_223	2008-04-09	11
+223	val_223	2008-04-09	11
+224	val_224	2008-04-09	11
+224	val_224	2008-04-09	11
+226	val_226	2008-04-09	11
+228	val_228	2008-04-09	11
+229	val_229	2008-04-09	11
+229	val_229	2008-04-09	11
+230	val_230	2008-04-09	11
+230	val_230	2008-04-09	11
+230	val_230	2008-04-09	11
+230	val_230	2008-04-09	11
+230	val_230	2008-04-09	11
+233	val_233	2008-04-09	11
+233	val_233	2008-04-09	11
+235	val_235	2008-04-09	11
+237	val_237	2008-04-09	11
+237	val_237	2008-04-09	11
+238	val_238	2008-04-09	11
+238	val_238	2008-04-09	11
+239	val_239	2008-04-09	11
+239	val_239	2008-04-09	11
+241	val_241	2008-04-09	11
+242	val_242	2008-04-09	11
+242	val_242	2008-04-09	11
+244	val_244	2008-04-09	11
+247	val_247	2008-04-09	11
+248	val_248	2008-04-09	11
+249	val_249	2008-04-09	11
+252	val_252	2008-04-09	11
+255	val_255	2008-04-09	11
+255	val_255	2008-04-09	11
+256	val_256	2008-04-09	11
+256	val_256	2008-04-09	11
+257	val_257	2008-04-09	11
+258	val_258	2008-04-09	11
+260	val_260	2008-04-09	11
+262	val_262	2008-04-09	11
+263	val_263	2008-04-09	11
+265	val_265	2008-04-09	11
+265	val_265	2008-04-09	11
+266	val_266	2008-04-09	11
+272	val_272	2008-04-09	11
+272	val_272	2008-04-09	11
+273	val_273	2008-04-09	11
+273	val_273	2008-04-09	11
+273	val_273	2008-04-09	11
+274	val_274	2008-04-09	11
+275	val_275	2008-04-09	11
+277	val_277	2008-04-09	11
+277	val_277	2008-04-09	11
+277	val_277	2008-04-09	11
+277	val_277	2008-04-09	11
+278	val_278	2008-04-09	11
+278	val_278	2008-04-09	11
+280	val_280	2008-04-09	11
+280	val_280	2008-04-09	11
+281	val_281	2008-04-09	11
+281	val_281	2008-04-09	11
+282	val_282	2008-04-09	11
+282	val_282	2008-04-09	11
+283	val_283	2008-04-09	11
+284	val_284	2008-04-09	11
+285	val_285	2008-04-09	11
+286	val_286	2008-04-09	11
+287	val_287	2008-04-09	11
+288	val_288	2008-04-09	11
+288	val_288	2008-04-09	11
+289	val_289	2008-04-09	11
+291	val_291	2008-04-09	11
+292	val_292	2008-04-09	11
+296	val_296	2008-04-09	11
+298	val_298	2008-04-09	11
+298	val_298	2008-04-09	11
+298	val_298	2008-04-09	11
+302	val_302	2008-04-09	11
+305	val_305	2008-04-09	11
+306	val_306	2008-04-09	11
+307	val_307	2008-04-09	11
+307	val_307	2008-04-09	11
+308	val_308	2008-04-09	11
+309	val_309	2008-04-09	11
+309	val_309	2008-04-09	11
+310	val_310	2008-04-09	11
+311	val_311	2008-04-09	11
+311	val_311	2008-04-09	11
+311	val_311	2008-04-09	11
+315	val_315	2008-04-09	11
+316	val_316	2008-04-09	11
+316	val_316	2008-04-09	11
+316	val_316	2008-04-09	11
+317	val_317	2008-04-09	11
+317	val_317	2008-04-09	11
+318	val_318	2008-04-09	11
+318	val_318	2008-04-09	11
+318	val_318	2008-04-09	11
+321	val_321	2008-04-09	11
+321	val_321	2008-04-09	11
+322	val_322	2008-04-09	11
+322	val_322	2008-04-09	11
+323	val_323	2008-04-09	11
+325	val_325	2008-04-09	11
+325	val_325	2008-04-09	11
+327	val_327	2008-04-09	11
+327	val_327	2008-04-09	11
+327	val_327	2008-04-09	11
+331	val_331	2008-04-09	11
+331	val_331	2008-04-09	11
+332	val_332	2008-04-09	11
+333	val_333	2008-04-09	11
+333	val_333	2008-04-09	11
+335	val_335	2008-04-09	11
+336	val_336	2008-04-09	11
+338	val_338	2008-04-09	11
+339	val_339	2008-04-09	11
+341	val_341	2008-04-09	11
+342	val_342	2008-04-09	11
+342	val_342	2008-04-09	11
+344	val_344	2008-04-09	11
+344	val_344	2008-04-09	11
+345	val_345	2008-04-09	11
+348	val_348	2008-04-09	11
+348	val_348	2008-04-09	11
+348	val_348	2008-04-09	11
+348	val_348	2008-04-09	11
+348	val_348	2008-04-09	11
+351	val_351	2008-04-09	11
+353	val_353	2008-04-09	11
+353	val_353	2008-04-09	11
+356	val_356	2008-04-09	11
+360	val_360	2008-04-09	11
+362	val_362	2008-04-09	11
+364	val_364	2008-04-09	11
+365	val_365	2008-04-09	11
+366	val_366	2008-04-09	11
+367	val_367	2008-04-09	11
+367	val_367	2008-04-09	11
+368	val_368	2008-04-09	11
+369	val_369	2008-04-09	11
+369	val_369	2008-04-09	11
+369	val_369	2008-04-09	11
+373	val_373	2008-04-09	11
+374	val_374	2008-04-09	11
+375	val_375	2008-04-09	11
+377	val_377	2008-04-09	11
+378	val_378	2008-04-09	11
+379	val_379	2008-04-09	11
+382	val_382	2008-04-09	11
+382	val_382	2008-04-09	11
+384	val_384	2008-04-09	11
+384	val_384	2008-04-09	11
+384	val_384	2008-04-09	11
+386	val_386	2008-04-09	11
+389	val_389	2008-04-09	11
+392	val_392	2008-04-09	11
+393	val_393	2008-04-09	11
+394	val_394	2008-04-09	11
+395	val_395	2008-04-09	11
+395	val_395	2008-04-09	11
+396	val_396	2008-04-09	11
+396	val_396	2008-04-09	11
+396	val_396	2008-04-09	11
+397	val_397	2008-04-09	11
+397	val_397	2008-04-09	11
+399	val_399	2008-04-09	11
+399	val_399	2008-04-09	11
+400	val_400	2008-04-09	11
+401	val_401	2008-04-09	11
+401	val_401	2008-04-09	11
+401	val_401	2008-04-09	11
+401	val_401	2008-04-09	11
+401	val_401	2008-04-09	11
+402	val_402	2008-04-09	11
+403	val_403	2008-04-09	11
+403	val_403	2008-04-09	11
+403	val_403	2008-04-09	11
+404	val_404	2008-04-09	11
+404	val_404	2008-04-09	11
+406	val_406	2008-04-09	11
+406	val_406	2008-04-09	11
+406	val_406	2008-04-09	11
+406	val_406	2008-04-09	11
+407	val_407	2008-04-09	11
+409	val_409	2008-04-09	11
+409	val_409	2008-04-09	11
+409	val_409	2008-04-09	11
+411	val_411	2008-04-09	11
+413	val_413	2008-04-09	11
+413	val_413	2008-04-09	11
+414	val_414	2008-04-09	11
+414	val_414	2008-04-09	11
+417	val_417	2008-04-09	11
+417	val_417	2008-04-09	11
+417	val_417	2008-04-09	11
+418	val_418	2008-04-09	11
+419	val_419	2008-04-09	11
+421	val_421	2008-04-09	11
+424	val_424	2008-04-09	11
+424	val_424	2008-04-09	11
+427	val_427	2008-04-09	11
+429	val_429	2008-04-09	11
+429	val_429	2008-04-09	11
+430	val_430	2008-04-09	11
+430	val_430	2008-04-09	11
+430	val_430	2008-04-09	11
+431	val_431	2008-04-09	11
+431	val_431	2008-04-09	11
+431	val_431	2008-04-09	11
+432	val_432	2008-04-09	11
+435	val_435	2008-04-09	11
+436	val_436	2008-04-09	11
+437	val_437	2008-04-09	11
+438	val_438	2008-04-09	11
+438	val_438	2008-04-09	11
+438	val_438	2008-04-09	11
+439	val_439	2008-04-09	11
+439	val_439	2008-04-09	11
+443	val_443	2008-04-09	11
+444	val_444	2008-04-09	11
+446	val_446	2008-04-09	11
+448	val_448	2008-04-09	11
+449	val_449	2008-04-09	11
+452	val_452	2008-04-09	11
+453	val_453	2008-04-09	11
+454	val_454	2008-04-09	11
+454	val_454	2008-04-09	11
+454	val_454	2008-04-09	11
+455	val_455	2008-04-09	11
+457	val_457	2008-04-09	11
+458	val_458	2008-04-09	11
+458	val_458	2008-04-09	11
+459	val_459	2008-04-09	11
+459	val_459	2008-04-09	11
+460	val_460	2008-04-09	11
+462	val_462	2008-04-09	11
+462	val_462	2008-04-09	11
+463	val_463	2008-04-09	11
+463	val_463	2008-04-09	11
+466	val_466	2008-04-09	11
+466	val_466	2008-04-09	11
+466	val_466	2008-04-09	11
+467	val_467	2008-04-09	11
+468	val_468	2008-04-09	11
+468	val_468	2008-04-09	11
+468	val_468	2008-04-09	11
+468	val_468	2008-04-09	11
+469	val_469	2008-04-09	11
+469	val_469	2008-04-09	11
+469	val_469	2008-04-09	11
+469	val_469	2008-04-09	11
+469	val_469	2008-04-09	11
+470	val_470	2008-04-09	11
+472	val_472	2008-04-09	11
+475	val_475	2008-04-09	11
+477	val_477	2008-04-09	11
+478	val_478	2008-04-09	11
+478	val_478	2008-04-09	11
+479	val_479	2008-04-09	11
+480	val_480	2008-04-09	11
+480	val_480	2008-04-09	11
+480	val_480	2008-04-09	11
+481	val_481	2008-04-09	11
+482	val_482	2008-04-09	11
+483	val_483	2008-04-09	11
+484	val_484	2008-04-09	11
+485	val_485	2008-04-09	11
+487	val_487	2008-04-09	11
+489	val_489	2008-04-09	11
+489	val_489	2008-04-09	11
+489	val_489	2008-04-09	11
+489	val_489	2008-04-09	11
+490	val_490	2008-04-09	11
+491	val_491	2008-04-09	11
+492	val_492	2008-04-09	11
+492	val_492	2008-04-09	11
+493	val_493	2008-04-09	11
+494	val_494	2008-04-09	11
+495	val_495	2008-04-09	11
+496	val_496	2008-04-09	11
+497	val_497	2008-04-09	11
+498	val_498	2008-04-09	11
+498	val_498	2008-04-09	11
+498	val_498	2008-04-09	11
+0	val_0	2008-04-09	12
+0	val_0	2008-04-09	12
+0	val_0	2008-04-09	12
+2	val_2	2008-04-09	12
+4	val_4	2008-04-09	12
+5	val_5	2008-04-09	12
+5	val_5	2008-04-09	12
+5	val_5	2008-04-09	12
+8	val_8	2008-04-09	12
+9	val_9	2008-04-09	12
+10	val_10	2008-04-09	12
+11	val_11	2008-04-09	12
+12	val_12	2008-04-09	12
+12	val_12	2008-04-09	12
+15	val_15	2008-04-09	12
+15	val_15	2008-04-09	12
+17	val_17	2008-04-09	12
+18	val_18	2008-04-09	12
+18	val_18	2008-04-09	12
+19	val_19	2008-04-09	12
+20	val_20	2008-04-09	12
+24	val_24	2008-04-09	12
+24	val_24	2008-04-09	12
+26	val_26	2008-04-09	12
+26	val_26	2008-04-09	12
+27	val_27	2008-04-09	12
+28	val_28	2008-04-09	12
+30	val_30	2008-04-09	12
+33	val_33	2008-04-09	12
+34	val_34	2008-04-09	12
+35	val_35	2008-04-09	12
+35	val_35	2008-04-09	12
+35	val_35	2008-04-09	12
+37	val_37	2008-04-09	12
+37	val_37	2008-04-09	12
+41	val_41	2008-04-09	12
+42	val_42	2008-04-09	12
+42	val_42	2008-04-09	12
+43	val_43	2008-04-09	12
+44	val_44	2008-04-09	12
+47	val_47	2008-04-09	12
+51	val_51	2008-04-09	12
+51	val_51	2008-04-09	12
+53	val_53	2008-04-09	12
+54	val_54	2008-04-09	12
+57	val_57	2008-04-09	12
+58	val_58	2008-04-09	12
+58	val_58	2008-04-09	12
+64	val_64	2008-04-09	12
+65	val_65	2008-04-09	12
+66	val_66	2008-04-09	12
+67	val_67	2008-04-09	12
+67	val_67	2008-04-09	12
+69	val_69	2008-04-09	12
+70	val_70	2008-04-09	12
+70	val_70	2008-04-09	12
+70	val_70	2008-04-09	12
+72	val_72	2008-04-09	12
+72	val_72	2008-04-09	12
+74	val_74	2008-04-09	12
+76	val_76	2008-04-09	12
+76	val_76	2008-04-09	12
+77	val_77	2008-04-09	12
+78	val_78	2008-04-09	12
+80	val_80	2008-04-09	12
+82	val_82	2008-04-09	12
+83	val_83	2008-04-09	12
+83	val_83	2008-04-09	12
+84	val_84	2008-04-09	12
+84	val_84	2008-04-09	12
+85	val_85	2008-04-09	12
+86	val_86	2008-04-09	12
+87	val_87	2008-04-09	12
+90	val_90	2008-04-09	12
+90	val_90	2008-04-09	12
+90	val_90	2008-04-09	12
+92	val_92	2008-04-09	12
+95	val_95	2008-04-09	12
+95	val_95	2008-04-09	12
+96	val_96	2008-04-09	12
+97	val_97	2008-04-09	12
+97	val_97	2008-04-09	12
+98	val_98	2008-04-09	12
+98	val_98	2008-04-09	12
+100	val_100	2008-04-09	12
+100	val_100	2008-04-09	12
+103	val_103	2008-04-09	12
+103	val_103	2008-04-09	12
+104	val_104	2008-04-09	12
+104	val_104	2008-04-09	12
+105	val_105	2008-04-09	12
+111	val_111	2008-04-09	12
+113	val_113	2008-04-09	12
+113	val_113	2008-04-09	12
+114	val_114	2008-04-09	12
+116	val_116	2008-04-09	12
+118	val_118	2008-04-09	12
+118	val_118	2008-04-09	12
+119	val_119	2008-04-09	12
+119	val_119	2008-04-09	12
+119	val_119	2008-04-09	12
+120	val_120	2008-04-09	12
+120	val_120	2008-04-09	12
+125	val_125	2008-04-09	12
+125	val_125	2008-04-09	12
+126	val_126	2008-04-09	12
+128	val_128	2008-04-09	12
+128	val_128	2008-04-09	12
+128	val_128	2008-04-09	12
+129	val_129	2008-04-09	12
+129	val_129	2008-04-09	12
+131	val_131	2008-04-09	12
+133	val_133	2008-04-09	12
+134	val_134	2008-04-09	12
+134	val_134	2008-04-09	12
+136	val_136	2008-04-09	12
+137	val_137	2008-04-09	12
+137	val_137	2008-04-09	12
+138	val_138	2008-04-09	12
+138	val_138	2008-04-09	12
+138	val_138	2008-04-09	12
+138	val_138	2008-04-09	12
+143	val_143	2008-04-09	12
+145	val_145	2008-04-09	12
+146	val_146	2008-04-09	12
+146	val_146	2008-04-09	12
+149	val_149	2008-04-09	12
+149	val_149	2008-04-09	12
+150	val_150	2008-04-09	12
+152	val_152	2008-04-09	12
+152	val_152	2008-04-09	12
+153	val_153	2008-04-09	12
+155	val_155	2008-04-09	12
+156	val_156	2008-04-09	12
+157	val_157	2008-04-09	12
+158	val_158	2008-04-09	12
+160	val_160	2008-04-09	12
+162	val_162	2008-04-09	12
+163	val_163	2008-04-09	12
+164	val_164	2008-04-09	12
+164	val_164	2008-04-09	12
+165	val_165	2008-04-09	12
+165	val_165	2008-04-09	12
+166	val_166	2008-04-09	12
+167	val_167	2008-04-09	12
+167	val_167	2008-04-09	12
+167	val_167	2008-04-09	12
+168	val_168	2008-04-09	12
+169	val_169	2008-04-09	12
+169	val_169	2008-04-09	12
+169	val_169	2008-04-09	12
+169	val_169	2008-04-09	12
+170	val_170	2008-04-09	12
+172	val_172	2008-04-09	12
+172	val_172	2008-04-09	12
+174	val_174	2008-04-09	12
+174	val_174	2008-04-09	12
+175	val_175	2008-04-09	12
+175	val_175	2008-04-09	12
+176	val_176	2008-04-09	12
+176	val_176	2008-04-09	12
+177	val_177	2008-04-09	12
+178	val_178	2008-04-09	12
+179	val_179	2008-04-09	12
+179	val_179	2008-04-09	12
+180	val_180	2008-04-09	12
+181	val_181	2008-04-09	12
+183	val_183	2008-04-09	12
+186	val_186	2008-04-09	12
+187	val_187	2008-04-09	12
+187	val_187	2008-04-09	12
+187	val_187	2008-04-09	12
+189	val_189	2008-04-09	12
+190	val_190	2008-04-09	12
+191	val_191	2008-04-09	12
+191	val_191	2008-04-09	12
+192	val_192	2008-04-09	12
+193	val_193	2008-04-09	12
+193	val_193	2008-04-09	12
+193	val_193	2008-04-09	12
+194	val_194	2008-04-09	12
+195	val_195	2008-04-09	12
+195	val_195	2008-04-09	12
+196	val_196	2008-04-09	12
+197	val_197	2008-04-09	12
+197	val_197	2008-04-09	12
+199	val_199	2008-04-09	12
+199	val_199	2008-04-09	12
+199	val_199	2008-04-09	12
+200	val_200	2008-04-09	12
+200	val_200	2008-04-09	12
+201	val_201	2008-04-09	12
+202	val_202	2008-04-09	12
+203	val_203	2008-04-09	12
+203	val_203	2008-04-09	12
+205	val_205	2008-04-09	12
+205	val_205	2008-04-09	12
+207	val_207	2008-04-09	12
+207	val_207	2008-04-09	12
+208	val_208	2008-04-09	12
+208	val_208	2008-04-09	12
+208	val_208	2008-04-09	12
+209	val_209	2008-04-09	12
+209	val_209	2008-04-09	12
+213	val_213	2008-04-09	12
+213	val_213	2008-04-09	12
+214	val_214	2008-04-09	12
+216	val_216	2008-04-09	12
+216	val_216	2008-04-09	12
+217	val_217	2008-04-09	12
+217	val_217	2008-04-09	12
+218	val_218	2008-04-09	12
+219	val_219	2008-04-09	12
+219	val_219	2008-04-09	12
+221	val_221	2008-04-09	12
+221	val_221	2008-04-09	12
+222	val_222	2008-04-09	12
+223	val_223	2008-04-09	12
+223	val_223	2008-04-09	12
+224	val_224	2008-04-09	12
+224	val_224	2008-04-09	12
+226	val_226	2008-04-09	12
+228	val_228	2008-04-09	12
+229	val_229	2008-04-09	12
+229	val_229	2008-04-09	12
+230	val_230	2008-04-09	12
+230	val_230	2008-04-09	12
+230	val_230	2008-04-09	12
+230	val_230	2008-04-09	12
+230	val_230	2008-04-09	12
+233	val_233	2008-04-09	12
+233	val_233	2008-04-09	12
+235	val_235	2008-04-09	12
+237	val_237	2008-04-09	12
+237	val_237	2008-04-09	12
+238	val_238	2008-04-09	12
+238	val_238	2008-04-09	12
+239	val_239	2008-04-09	12
+239	val_239	2008-04-09	12
+241	val_241	2008-04-09	12
+242	val_242	2008-04-09	12
+242	val_242	2008-04-09	12
+244	val_244	2008-04-09	12
+247	val_247	2008-04-09	12
+248	val_248	2008-04-09	12
+249	val_249	2008-04-09	12
+252	val_252	2008-04-09	12
+255	val_255	2008-04-09	12
+255	val_255	2008-04-09	12
+256	val_256	2008-04-09	12
+256	val_256	2008-04-09	12
+257	val_257	2008-04-09	12
+258	val_258	2008-04-09	12
+260	val_260	2008-04-09	12
+262	val_262	2008-04-09	12
+263	val_263	2008-04-09	12
+265	val_265	2008-04-09	12
+265	val_265	2008-04-09	12
+266	val_266	2008-04-09	12
+272	val_272	2008-04-09	12
+272	val_272	2008-04-09	12
+273	val_273	2008-04-09	12
+273	val_273	2008-04-09	12
+273	val_273	2008-04-09	12
+274	val_274	2008-04-09	12
+275	val_275	2008-04-09	12
+277	val_277	2008-04-09	12
+277	val_277	2008-04-09	12
+277	val_277	2008-04-09	12
+277	val_277	2008-04-09	12
+278	val_278	2008-04-09	12
+278	val_278	2008-04-09	12
+280	val_280	2008-04-09	12
+280	val_280	2008-04-09	12
+281	val_281	2008-04-09	12
+281	val_281	2008-04-09	12
+282	val_282	2008-04-09	12
+282	val_282	2008-04-09	12
+283	val_283	2008-04-09	12
+284	val_284	2008-04-09	12
+285	val_285	2008-04-09	12
+286	val_286	2008-04-09	12
+287	val_287	2008-04-09	12
+288	val_288	2008-04-09	12
+288	val_288	2008-04-09	12
+289	val_289	2008-04-09	12
+291	val_291	2008-04-09	12
+292	val_292	2008-04-09	12
+296	val_296	2008-04-09	12
+298	val_298	2008-04-09	12
+298	val_298	2008-04-09	12
+298	val_298	2008-04-09	12
+302	val_302	2008-04-09	12
+305	val_305	2008-04-09	12
+306	val_306	2008-04-09	12
+307	val_307	2008-04-09	12
+307	val_307	2008-04-09	12
+308	val_308	2008-04-09	12
+309	val_309	2008-04-09	12
+309	val_309	2008-04-09	12
+310	val_310	2008-04-09	12
+311	val_311	2008-04-09	12
+311	val_311	2008-04-09	12
+311	val_311	2008-04-09	12
+315	val_315	2008-04-09	12
+316	val_316	2008-04-09	12
+316	val_316	2008-04-09	12
+316	val_316	2008-04-09	12
+317	val_317	2008-04-09	12
+317	val_317	2008-04-09	12
+318	val_318	2008-04-09	12
+318	val_318	2008-04-09	12
+318	val_318	2008-04-09	12
+321	val_321	2008-04-09	12
+321	val_321	2008-04-09	12
+322	val_322	2008-04-09	12
+322	val_322	2008-04-09	12
+323	val_323	2008-04-09	12
+325	val_325	2008-04-09	12
+325	val_325	2008-04-09	12
+327	val_327	2008-04-09	12
+327	val_327	2008-04-09	12
+327	val_327	2008-04-09	12
+331	val_331	2008-04-09	12
+331	val_331	2008-04-09	12
+332	val_332	2008-04-09	12
+333	val_333	2008-04-09	12
+333	val_333	2008-04-09	12
+335	val_335	2008-04-09	12
+336	val_336	2008-04-09	12
+338	val_338	2008-04-09	12
+339	val_339	2008-04-09	12
+341	val_341	2008-04-09	12
+342	val_342	2008-04-09	12
+342	val_342	2008-04-09	12
+344	val_344	2008-04-09	12
+344	val_344	2008-04-09	12
+345	val_345	2008-04-09	12
+348	val_348	2008-04-09	12
+348	val_348	2008-04-09	12
+348	val_348	2008-04-09	12
+348	val_348	2008-04-09	12
+348	val_348	2008-04-09	12
+351	val_351	2008-04-09	12
+353	val_353	2008-04-09	12
+353	val_353	2008-04-09	12
+356	val_356	2008-04-09	12
+360	val_360	2008-04-09	12
+362	val_362	2008-04-09	12
+364	val_364	2008-04-09	12
+365	val_365	2008-04-09	12
+366	val_366	2008-04-09	12
+367	val_367	2008-04-09	12
+367	val_367	2008-04-09	12
+368	val_368	2008-04-09	12
+369	val_369	2008-04-09	12
+369	val_369	2008-04-09	12
+369	val_369	2008-04-09	12
+373	val_373	2008-04-09	12
+374	val_374	2008-04-09	12
+375	val_375	2008-04-09	12
+377	val_377	2008-04-09	12
+378	val_378	2008-04-09	12
+379	val_379	2008-04-09	12
+382	val_382	2008-04-09	12
+382	val_382	2008-04-09	12
+384	val_384	2008-04-09	12
+384	val_384	2008-04-09	12
+384	val_384	2008-04-09	12
+386	val_386	2008-04-09	12
+389	val_389	2008-04-09	12
+392	val_392	2008-04-09	12
+393	val_393	2008-04-09	12
+394	val_394	2008-04-09	12
+395	val_395	2008-04-09	12
+395	val_395	2008-04-09	12
+396	val_396	2008-04-09	12
+396	val_396	2008-04-09	12
+396	val_396	2008-04-09	12
+397	val_397	2008-04-09	12
+397	val_397	2008-04-09	12
+399	val_399	2008-04-09	12
+399	val_399	2008-04-09	12
+400	val_400	2008-04-09	12
+401	val_401	2008-04-09	12
+401	val_401	2008-04-09	12
+401	val_401	2008-04-09	12
+401	val_401	2008-04-09	12
+401	val_401	2008-04-09	12
+402	val_402	2008-04-09	12
+403	val_403	2008-04-09	12
+403	val_403	2008-04-09	12
+403	val_403	2008-04-09	12
+404	val_404	2008-04-09	12
+404	val_404	2008-04-09	12
+406	val_406	2008-04-09	12
+406	val_406	2008-04-09	12
+406	val_406	2008-04-09	12
+406	val_406	2008-04-09	12
+407	val_407	2008-04-09	12
+409	val_409	2008-04-09	12
+409	val_409	2008-04-09	12
+409	val_409	2008-04-09	12
+411	val_411	2008-04-09	12
+413	val_413	2008-04-09	12
+413	val_413	2008-04-09	12
+414	val_414	2008-04-09	12
+414	val_414	2008-04-09	12
+417	val_417	2008-04-09	12
+417	val_417	2008-04-09	12
+417	val_417	2008-04-09	12
+418	val_418	2008-04-09	12
+419	val_419	2008-04-09	12
+421	val_421	2008-04-09	12
+424	val_424	2008-04-09	12
+424	val_424	2008-04-09	12
+427	val_427	2008-04-09	12
+429	val_429	2008-04-09	12
+429	val_429	2008-04-09	12
+430	val_430	2008-04-09	12
+430	val_430	2008-04-09	12
+430	val_430	2008-04-09	12
+431	val_431	2008-04-09	12
+431	val_431	2008-04-09	12
+431	val_431	2008-04-09	12
+432	val_432	2008-04-09	12
+435	val_435	2008-04-09	12
+436	val_436	2008-04-09	12
+437	val_437	2008-04-09	12
+438	val_438	2008-04-09	12
+438	val_438	2008-04-09	12
+438	val_438	2008-04-09	12
+439	val_439	2008-04-09	12
+439	val_439	2008-04-09	12
+443	val_443	2008-04-09	12
+444	val_444	2008-04-09	12
+446	val_446	2008-04-09	12
+448	val_448	2008-04-09	12
+449	val_449	2008-04-09	12
+452	val_452	2008-04-09	12
+453	val_453	2008-04-09	12
+454	val_454	2008-04-09	12
+454	val_454	2008-04-09	12
+454	val_454	2008-04-09	12
+455	val_455	2008-04-09	12
+457	val_457	2008-04-09	12
+458	val_458	2008-04-09	12
+458	val_458	2008-04-09	12
+459	val_459	2008-04-09	12
+459	val_459	2008-04-09	12
+460	val_460	2008-04-09	12
+462	val_462	2008-04-09	12
+462	val_462	2008-04-09	12
+463	val_463	2008-04-09	12
+463	val_463	2008-04-09	12
+466	val_466	2008-04-09	12
+466	val_466	2008-04-09	12
+466	val_466	2008-04-09	12
+467	val_467	2008-04-09	12
+468	val_468	2008-04-09	12
+468	val_468	2008-04-09	12
+468	val_468	2008-04-09	12
+468	val_468	2008-04-09	12
+469	val_469	2008-04-09	12
+469	val_469	2008-04-09	12
+469	val_469	2008-04-09	12
+469	val_469	2008-04-09	12
+469	val_469	2008-04-09	12
+470	val_470	2008-04-09	12
+472	val_472	2008-04-09	12
+475	val_475	2008-04-09	12
+477	val_477	2008-04-09	12
+478	val_478	2008-04-09	12
+478	val_478	2008-04-09	12
+479	val_479	2008-04-09	12
+480	val_480	2008-04-09	12
+480	val_480	2008-04-09	12
+480	val_480	2008-04-09	12
+481	val_481	2008-04-09	12
+482	val_482	2008-04-09	12
+483	val_483	2008-04-09	12
+484	val_484	2008-04-09	12
+485	val_485	2008-04-09	12
+487	val_487	2008-04-09	12
+489	val_489	2008-04-09	12
+489	val_489	2008-04-09	12
+489	val_489	2008-04-09	12
+489	val_489	2008-04-09	12
+490	val_490	2008-04-09	12
+491	val_491	2008-04-09	12
+492	val_492	2008-04-09	12
+492	val_492	2008-04-09	12
+493	val_493	2008-04-09	12
+494	val_494	2008-04-09	12
+495	val_495	2008-04-09	12
+496	val_496	2008-04-09	12
+497	val_497	2008-04-09	12
+498	val_498	2008-04-09	12
+498	val_498	2008-04-09	12
+498	val_498	2008-04-09	12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 b/sql/hive/src/test/resources/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437
deleted file mode 100644
index 2857cdf0aba86636e320800d63c5ebf3f5975b08..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437
+++ /dev/null
@@ -1,2000 +0,0 @@
-238	val_238	2008-04-08	11
-86	val_86	2008-04-08	11
-311	val_311	2008-04-08	11
-27	val_27	2008-04-08	11
-165	val_165	2008-04-08	11
-409	val_409	2008-04-08	11
-255	val_255	2008-04-08	11
-278	val_278	2008-04-08	11
-98	val_98	2008-04-08	11
-484	val_484	2008-04-08	11
-265	val_265	2008-04-08	11
-193	val_193	2008-04-08	11
-401	val_401	2008-04-08	11
-150	val_150	2008-04-08	11
-273	val_273	2008-04-08	11
-224	val_224	2008-04-08	11
-369	val_369	2008-04-08	11
-66	val_66	2008-04-08	11
-128	val_128	2008-04-08	11
-213	val_213	2008-04-08	11
-146	val_146	2008-04-08	11
-406	val_406	2008-04-08	11
-429	val_429	2008-04-08	11
-374	val_374	2008-04-08	11
-152	val_152	2008-04-08	11
-469	val_469	2008-04-08	11
-145	val_145	2008-04-08	11
-495	val_495	2008-04-08	11
-37	val_37	2008-04-08	11
-327	val_327	2008-04-08	11
-281	val_281	2008-04-08	11
-277	val_277	2008-04-08	11
-209	val_209	2008-04-08	11
-15	val_15	2008-04-08	11
-82	val_82	2008-04-08	11
-403	val_403	2008-04-08	11
-166	val_166	2008-04-08	11
-417	val_417	2008-04-08	11
-430	val_430	2008-04-08	11
-252	val_252	2008-04-08	11
-292	val_292	2008-04-08	11
-219	val_219	2008-04-08	11
-287	val_287	2008-04-08	11
-153	val_153	2008-04-08	11
-193	val_193	2008-04-08	11
-338	val_338	2008-04-08	11
-446	val_446	2008-04-08	11
-459	val_459	2008-04-08	11
-394	val_394	2008-04-08	11
-237	val_237	2008-04-08	11
-482	val_482	2008-04-08	11
-174	val_174	2008-04-08	11
-413	val_413	2008-04-08	11
-494	val_494	2008-04-08	11
-207	val_207	2008-04-08	11
-199	val_199	2008-04-08	11
-466	val_466	2008-04-08	11
-208	val_208	2008-04-08	11
-174	val_174	2008-04-08	11
-399	val_399	2008-04-08	11
-396	val_396	2008-04-08	11
-247	val_247	2008-04-08	11
-417	val_417	2008-04-08	11
-489	val_489	2008-04-08	11
-162	val_162	2008-04-08	11
-377	val_377	2008-04-08	11
-397	val_397	2008-04-08	11
-309	val_309	2008-04-08	11
-365	val_365	2008-04-08	11
-266	val_266	2008-04-08	11
-439	val_439	2008-04-08	11
-342	val_342	2008-04-08	11
-367	val_367	2008-04-08	11
-325	val_325	2008-04-08	11
-167	val_167	2008-04-08	11
-195	val_195	2008-04-08	11
-475	val_475	2008-04-08	11
-17	val_17	2008-04-08	11
-113	val_113	2008-04-08	11
-155	val_155	2008-04-08	11
-203	val_203	2008-04-08	11
-339	val_339	2008-04-08	11
-0	val_0	2008-04-08	11
-455	val_455	2008-04-08	11
-128	val_128	2008-04-08	11
-311	val_311	2008-04-08	11
-316	val_316	2008-04-08	11
-57	val_57	2008-04-08	11
-302	val_302	2008-04-08	11
-205	val_205	2008-04-08	11
-149	val_149	2008-04-08	11
-438	val_438	2008-04-08	11
-345	val_345	2008-04-08	11
-129	val_129	2008-04-08	11
-170	val_170	2008-04-08	11
-20	val_20	2008-04-08	11
-489	val_489	2008-04-08	11
-157	val_157	2008-04-08	11
-378	val_378	2008-04-08	11
-221	val_221	2008-04-08	11
-92	val_92	2008-04-08	11
-111	val_111	2008-04-08	11
-47	val_47	2008-04-08	11
-72	val_72	2008-04-08	11
-4	val_4	2008-04-08	11
-280	val_280	2008-04-08	11
-35	val_35	2008-04-08	11
-427	val_427	2008-04-08	11
-277	val_277	2008-04-08	11
-208	val_208	2008-04-08	11
-356	val_356	2008-04-08	11
-399	val_399	2008-04-08	11
-169	val_169	2008-04-08	11
-382	val_382	2008-04-08	11
-498	val_498	2008-04-08	11
-125	val_125	2008-04-08	11
-386	val_386	2008-04-08	11
-437	val_437	2008-04-08	11
-469	val_469	2008-04-08	11
-192	val_192	2008-04-08	11
-286	val_286	2008-04-08	11
-187	val_187	2008-04-08	11
-176	val_176	2008-04-08	11
-54	val_54	2008-04-08	11
-459	val_459	2008-04-08	11
-51	val_51	2008-04-08	11
-138	val_138	2008-04-08	11
-103	val_103	2008-04-08	11
-239	val_239	2008-04-08	11
-213	val_213	2008-04-08	11
-216	val_216	2008-04-08	11
-430	val_430	2008-04-08	11
-278	val_278	2008-04-08	11
-176	val_176	2008-04-08	11
-289	val_289	2008-04-08	11
-221	val_221	2008-04-08	11
-65	val_65	2008-04-08	11
-318	val_318	2008-04-08	11
-332	val_332	2008-04-08	11
-311	val_311	2008-04-08	11
-275	val_275	2008-04-08	11
-137	val_137	2008-04-08	11
-241	val_241	2008-04-08	11
-83	val_83	2008-04-08	11
-333	val_333	2008-04-08	11
-180	val_180	2008-04-08	11
-284	val_284	2008-04-08	11
-12	val_12	2008-04-08	11
-230	val_230	2008-04-08	11
-181	val_181	2008-04-08	11
-67	val_67	2008-04-08	11
-260	val_260	2008-04-08	11
-404	val_404	2008-04-08	11
-384	val_384	2008-04-08	11
-489	val_489	2008-04-08	11
-353	val_353	2008-04-08	11
-373	val_373	2008-04-08	11
-272	val_272	2008-04-08	11
-138	val_138	2008-04-08	11
-217	val_217	2008-04-08	11
-84	val_84	2008-04-08	11
-348	val_348	2008-04-08	11
-466	val_466	2008-04-08	11
-58	val_58	2008-04-08	11
-8	val_8	2008-04-08	11
-411	val_411	2008-04-08	11
-230	val_230	2008-04-08	11
-208	val_208	2008-04-08	11
-348	val_348	2008-04-08	11
-24	val_24	2008-04-08	11
-463	val_463	2008-04-08	11
-431	val_431	2008-04-08	11
-179	val_179	2008-04-08	11
-172	val_172	2008-04-08	11
-42	val_42	2008-04-08	11
-129	val_129	2008-04-08	11
-158	val_158	2008-04-08	11
-119	val_119	2008-04-08	11
-496	val_496	2008-04-08	11
-0	val_0	2008-04-08	11
-322	val_322	2008-04-08	11
-197	val_197	2008-04-08	11
-468	val_468	2008-04-08	11
-393	val_393	2008-04-08	11
-454	val_454	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-199	val_199	2008-04-08	11
-191	val_191	2008-04-08	11
-418	val_418	2008-04-08	11
-96	val_96	2008-04-08	11
-26	val_26	2008-04-08	11
-165	val_165	2008-04-08	11
-327	val_327	2008-04-08	11
-230	val_230	2008-04-08	11
-205	val_205	2008-04-08	11
-120	val_120	2008-04-08	11
-131	val_131	2008-04-08	11
-51	val_51	2008-04-08	11
-404	val_404	2008-04-08	11
-43	val_43	2008-04-08	11
-436	val_436	2008-04-08	11
-156	val_156	2008-04-08	11
-469	val_469	2008-04-08	11
-468	val_468	2008-04-08	11
-308	val_308	2008-04-08	11
-95	val_95	2008-04-08	11
-196	val_196	2008-04-08	11
-288	val_288	2008-04-08	11
-481	val_481	2008-04-08	11
-457	val_457	2008-04-08	11
-98	val_98	2008-04-08	11
-282	val_282	2008-04-08	11
-197	val_197	2008-04-08	11
-187	val_187	2008-04-08	11
-318	val_318	2008-04-08	11
-318	val_318	2008-04-08	11
-409	val_409	2008-04-08	11
-470	val_470	2008-04-08	11
-137	val_137	2008-04-08	11
-369	val_369	2008-04-08	11
-316	val_316	2008-04-08	11
-169	val_169	2008-04-08	11
-413	val_413	2008-04-08	11
-85	val_85	2008-04-08	11
-77	val_77	2008-04-08	11
-0	val_0	2008-04-08	11
-490	val_490	2008-04-08	11
-87	val_87	2008-04-08	11
-364	val_364	2008-04-08	11
-179	val_179	2008-04-08	11
-118	val_118	2008-04-08	11
-134	val_134	2008-04-08	11
-395	val_395	2008-04-08	11
-282	val_282	2008-04-08	11
-138	val_138	2008-04-08	11
-238	val_238	2008-04-08	11
-419	val_419	2008-04-08	11
-15	val_15	2008-04-08	11
-118	val_118	2008-04-08	11
-72	val_72	2008-04-08	11
-90	val_90	2008-04-08	11
-307	val_307	2008-04-08	11
-19	val_19	2008-04-08	11
-435	val_435	2008-04-08	11
-10	val_10	2008-04-08	11
-277	val_277	2008-04-08	11
-273	val_273	2008-04-08	11
-306	val_306	2008-04-08	11
-224	val_224	2008-04-08	11
-309	val_309	2008-04-08	11
-389	val_389	2008-04-08	11
-327	val_327	2008-04-08	11
-242	val_242	2008-04-08	11
-369	val_369	2008-04-08	11
-392	val_392	2008-04-08	11
-272	val_272	2008-04-08	11
-331	val_331	2008-04-08	11
-401	val_401	2008-04-08	11
-242	val_242	2008-04-08	11
-452	val_452	2008-04-08	11
-177	val_177	2008-04-08	11
-226	val_226	2008-04-08	11
-5	val_5	2008-04-08	11
-497	val_497	2008-04-08	11
-402	val_402	2008-04-08	11
-396	val_396	2008-04-08	11
-317	val_317	2008-04-08	11
-395	val_395	2008-04-08	11
-58	val_58	2008-04-08	11
-35	val_35	2008-04-08	11
-336	val_336	2008-04-08	11
-95	val_95	2008-04-08	11
-11	val_11	2008-04-08	11
-168	val_168	2008-04-08	11
-34	val_34	2008-04-08	11
-229	val_229	2008-04-08	11
-233	val_233	2008-04-08	11
-143	val_143	2008-04-08	11
-472	val_472	2008-04-08	11
-322	val_322	2008-04-08	11
-498	val_498	2008-04-08	11
-160	val_160	2008-04-08	11
-195	val_195	2008-04-08	11
-42	val_42	2008-04-08	11
-321	val_321	2008-04-08	11
-430	val_430	2008-04-08	11
-119	val_119	2008-04-08	11
-489	val_489	2008-04-08	11
-458	val_458	2008-04-08	11
-78	val_78	2008-04-08	11
-76	val_76	2008-04-08	11
-41	val_41	2008-04-08	11
-223	val_223	2008-04-08	11
-492	val_492	2008-04-08	11
-149	val_149	2008-04-08	11
-449	val_449	2008-04-08	11
-218	val_218	2008-04-08	11
-228	val_228	2008-04-08	11
-138	val_138	2008-04-08	11
-453	val_453	2008-04-08	11
-30	val_30	2008-04-08	11
-209	val_209	2008-04-08	11
-64	val_64	2008-04-08	11
-468	val_468	2008-04-08	11
-76	val_76	2008-04-08	11
-74	val_74	2008-04-08	11
-342	val_342	2008-04-08	11
-69	val_69	2008-04-08	11
-230	val_230	2008-04-08	11
-33	val_33	2008-04-08	11
-368	val_368	2008-04-08	11
-103	val_103	2008-04-08	11
-296	val_296	2008-04-08	11
-113	val_113	2008-04-08	11
-216	val_216	2008-04-08	11
-367	val_367	2008-04-08	11
-344	val_344	2008-04-08	11
-167	val_167	2008-04-08	11
-274	val_274	2008-04-08	11
-219	val_219	2008-04-08	11
-239	val_239	2008-04-08	11
-485	val_485	2008-04-08	11
-116	val_116	2008-04-08	11
-223	val_223	2008-04-08	11
-256	val_256	2008-04-08	11
-263	val_263	2008-04-08	11
-70	val_70	2008-04-08	11
-487	val_487	2008-04-08	11
-480	val_480	2008-04-08	11
-401	val_401	2008-04-08	11
-288	val_288	2008-04-08	11
-191	val_191	2008-04-08	11
-5	val_5	2008-04-08	11
-244	val_244	2008-04-08	11
-438	val_438	2008-04-08	11
-128	val_128	2008-04-08	11
-467	val_467	2008-04-08	11
-432	val_432	2008-04-08	11
-202	val_202	2008-04-08	11
-316	val_316	2008-04-08	11
-229	val_229	2008-04-08	11
-469	val_469	2008-04-08	11
-463	val_463	2008-04-08	11
-280	val_280	2008-04-08	11
-2	val_2	2008-04-08	11
-35	val_35	2008-04-08	11
-283	val_283	2008-04-08	11
-331	val_331	2008-04-08	11
-235	val_235	2008-04-08	11
-80	val_80	2008-04-08	11
-44	val_44	2008-04-08	11
-193	val_193	2008-04-08	11
-321	val_321	2008-04-08	11
-335	val_335	2008-04-08	11
-104	val_104	2008-04-08	11
-466	val_466	2008-04-08	11
-366	val_366	2008-04-08	11
-175	val_175	2008-04-08	11
-403	val_403	2008-04-08	11
-483	val_483	2008-04-08	11
-53	val_53	2008-04-08	11
-105	val_105	2008-04-08	11
-257	val_257	2008-04-08	11
-406	val_406	2008-04-08	11
-409	val_409	2008-04-08	11
-190	val_190	2008-04-08	11
-406	val_406	2008-04-08	11
-401	val_401	2008-04-08	11
-114	val_114	2008-04-08	11
-258	val_258	2008-04-08	11
-90	val_90	2008-04-08	11
-203	val_203	2008-04-08	11
-262	val_262	2008-04-08	11
-348	val_348	2008-04-08	11
-424	val_424	2008-04-08	11
-12	val_12	2008-04-08	11
-396	val_396	2008-04-08	11
-201	val_201	2008-04-08	11
-217	val_217	2008-04-08	11
-164	val_164	2008-04-08	11
-431	val_431	2008-04-08	11
-454	val_454	2008-04-08	11
-478	val_478	2008-04-08	11
-298	val_298	2008-04-08	11
-125	val_125	2008-04-08	11
-431	val_431	2008-04-08	11
-164	val_164	2008-04-08	11
-424	val_424	2008-04-08	11
-187	val_187	2008-04-08	11
-382	val_382	2008-04-08	11
-5	val_5	2008-04-08	11
-70	val_70	2008-04-08	11
-397	val_397	2008-04-08	11
-480	val_480	2008-04-08	11
-291	val_291	2008-04-08	11
-24	val_24	2008-04-08	11
-351	val_351	2008-04-08	11
-255	val_255	2008-04-08	11
-104	val_104	2008-04-08	11
-70	val_70	2008-04-08	11
-163	val_163	2008-04-08	11
-438	val_438	2008-04-08	11
-119	val_119	2008-04-08	11
-414	val_414	2008-04-08	11
-200	val_200	2008-04-08	11
-491	val_491	2008-04-08	11
-237	val_237	2008-04-08	11
-439	val_439	2008-04-08	11
-360	val_360	2008-04-08	11
-248	val_248	2008-04-08	11
-479	val_479	2008-04-08	11
-305	val_305	2008-04-08	11
-417	val_417	2008-04-08	11
-199	val_199	2008-04-08	11
-444	val_444	2008-04-08	11
-120	val_120	2008-04-08	11
-429	val_429	2008-04-08	11
-169	val_169	2008-04-08	11
-443	val_443	2008-04-08	11
-323	val_323	2008-04-08	11
-325	val_325	2008-04-08	11
-277	val_277	2008-04-08	11
-230	val_230	2008-04-08	11
-478	val_478	2008-04-08	11
-178	val_178	2008-04-08	11
-468	val_468	2008-04-08	11
-310	val_310	2008-04-08	11
-317	val_317	2008-04-08	11
-333	val_333	2008-04-08	11
-493	val_493	2008-04-08	11
-460	val_460	2008-04-08	11
-207	val_207	2008-04-08	11
-249	val_249	2008-04-08	11
-265	val_265	2008-04-08	11
-480	val_480	2008-04-08	11
-83	val_83	2008-04-08	11
-136	val_136	2008-04-08	11
-353	val_353	2008-04-08	11
-172	val_172	2008-04-08	11
-214	val_214	2008-04-08	11
-462	val_462	2008-04-08	11
-233	val_233	2008-04-08	11
-406	val_406	2008-04-08	11
-133	val_133	2008-04-08	11
-175	val_175	2008-04-08	11
-189	val_189	2008-04-08	11
-454	val_454	2008-04-08	11
-375	val_375	2008-04-08	11
-401	val_401	2008-04-08	11
-421	val_421	2008-04-08	11
-407	val_407	2008-04-08	11
-384	val_384	2008-04-08	11
-256	val_256	2008-04-08	11
-26	val_26	2008-04-08	11
-134	val_134	2008-04-08	11
-67	val_67	2008-04-08	11
-384	val_384	2008-04-08	11
-379	val_379	2008-04-08	11
-18	val_18	2008-04-08	11
-462	val_462	2008-04-08	11
-492	val_492	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-9	val_9	2008-04-08	11
-341	val_341	2008-04-08	11
-498	val_498	2008-04-08	11
-146	val_146	2008-04-08	11
-458	val_458	2008-04-08	11
-362	val_362	2008-04-08	11
-186	val_186	2008-04-08	11
-285	val_285	2008-04-08	11
-348	val_348	2008-04-08	11
-167	val_167	2008-04-08	11
-18	val_18	2008-04-08	11
-273	val_273	2008-04-08	11
-183	val_183	2008-04-08	11
-281	val_281	2008-04-08	11
-344	val_344	2008-04-08	11
-97	val_97	2008-04-08	11
-469	val_469	2008-04-08	11
-315	val_315	2008-04-08	11
-84	val_84	2008-04-08	11
-28	val_28	2008-04-08	11
-37	val_37	2008-04-08	11
-448	val_448	2008-04-08	11
-152	val_152	2008-04-08	11
-348	val_348	2008-04-08	11
-307	val_307	2008-04-08	11
-194	val_194	2008-04-08	11
-414	val_414	2008-04-08	11
-477	val_477	2008-04-08	11
-222	val_222	2008-04-08	11
-126	val_126	2008-04-08	11
-90	val_90	2008-04-08	11
-169	val_169	2008-04-08	11
-403	val_403	2008-04-08	11
-400	val_400	2008-04-08	11
-200	val_200	2008-04-08	11
-97	val_97	2008-04-08	11
-238	val_238	2008-04-08	12
-86	val_86	2008-04-08	12
-311	val_311	2008-04-08	12
-27	val_27	2008-04-08	12
-165	val_165	2008-04-08	12
-409	val_409	2008-04-08	12
-255	val_255	2008-04-08	12
-278	val_278	2008-04-08	12
-98	val_98	2008-04-08	12
-484	val_484	2008-04-08	12
-265	val_265	2008-04-08	12
-193	val_193	2008-04-08	12
-401	val_401	2008-04-08	12
-150	val_150	2008-04-08	12
-273	val_273	2008-04-08	12
-224	val_224	2008-04-08	12
-369	val_369	2008-04-08	12
-66	val_66	2008-04-08	12
-128	val_128	2008-04-08	12
-213	val_213	2008-04-08	12
-146	val_146	2008-04-08	12
-406	val_406	2008-04-08	12
-429	val_429	2008-04-08	12
-374	val_374	2008-04-08	12
-152	val_152	2008-04-08	12
-469	val_469	2008-04-08	12
-145	val_145	2008-04-08	12
-495	val_495	2008-04-08	12
-37	val_37	2008-04-08	12
-327	val_327	2008-04-08	12
-281	val_281	2008-04-08	12
-277	val_277	2008-04-08	12
-209	val_209	2008-04-08	12
-15	val_15	2008-04-08	12
-82	val_82	2008-04-08	12
-403	val_403	2008-04-08	12
-166	val_166	2008-04-08	12
-417	val_417	2008-04-08	12
-430	val_430	2008-04-08	12
-252	val_252	2008-04-08	12
-292	val_292	2008-04-08	12
-219	val_219	2008-04-08	12
-287	val_287	2008-04-08	12
-153	val_153	2008-04-08	12
-193	val_193	2008-04-08	12
-338	val_338	2008-04-08	12
-446	val_446	2008-04-08	12
-459	val_459	2008-04-08	12
-394	val_394	2008-04-08	12
-237	val_237	2008-04-08	12
-482	val_482	2008-04-08	12
-174	val_174	2008-04-08	12
-413	val_413	2008-04-08	12
-494	val_494	2008-04-08	12
-207	val_207	2008-04-08	12
-199	val_199	2008-04-08	12
-466	val_466	2008-04-08	12
-208	val_208	2008-04-08	12
-174	val_174	2008-04-08	12
-399	val_399	2008-04-08	12
-396	val_396	2008-04-08	12
-247	val_247	2008-04-08	12
-417	val_417	2008-04-08	12
-489	val_489	2008-04-08	12
-162	val_162	2008-04-08	12
-377	val_377	2008-04-08	12
-397	val_397	2008-04-08	12
-309	val_309	2008-04-08	12
-365	val_365	2008-04-08	12
-266	val_266	2008-04-08	12
-439	val_439	2008-04-08	12
-342	val_342	2008-04-08	12
-367	val_367	2008-04-08	12
-325	val_325	2008-04-08	12
-167	val_167	2008-04-08	12
-195	val_195	2008-04-08	12
-475	val_475	2008-04-08	12
-17	val_17	2008-04-08	12
-113	val_113	2008-04-08	12
-155	val_155	2008-04-08	12
-203	val_203	2008-04-08	12
-339	val_339	2008-04-08	12
-0	val_0	2008-04-08	12
-455	val_455	2008-04-08	12
-128	val_128	2008-04-08	12
-311	val_311	2008-04-08	12
-316	val_316	2008-04-08	12
-57	val_57	2008-04-08	12
-302	val_302	2008-04-08	12
-205	val_205	2008-04-08	12
-149	val_149	2008-04-08	12
-438	val_438	2008-04-08	12
-345	val_345	2008-04-08	12
-129	val_129	2008-04-08	12
-170	val_170	2008-04-08	12
-20	val_20	2008-04-08	12
-489	val_489	2008-04-08	12
-157	val_157	2008-04-08	12
-378	val_378	2008-04-08	12
-221	val_221	2008-04-08	12
-92	val_92	2008-04-08	12
-111	val_111	2008-04-08	12
-47	val_47	2008-04-08	12
-72	val_72	2008-04-08	12
-4	val_4	2008-04-08	12
-280	val_280	2008-04-08	12
-35	val_35	2008-04-08	12
-427	val_427	2008-04-08	12
-277	val_277	2008-04-08	12
-208	val_208	2008-04-08	12
-356	val_356	2008-04-08	12
-399	val_399	2008-04-08	12
-169	val_169	2008-04-08	12
-382	val_382	2008-04-08	12
-498	val_498	2008-04-08	12
-125	val_125	2008-04-08	12
-386	val_386	2008-04-08	12
-437	val_437	2008-04-08	12
-469	val_469	2008-04-08	12
-192	val_192	2008-04-08	12
-286	val_286	2008-04-08	12
-187	val_187	2008-04-08	12
-176	val_176	2008-04-08	12
-54	val_54	2008-04-08	12
-459	val_459	2008-04-08	12
-51	val_51	2008-04-08	12
-138	val_138	2008-04-08	12
-103	val_103	2008-04-08	12
-239	val_239	2008-04-08	12
-213	val_213	2008-04-08	12
-216	val_216	2008-04-08	12
-430	val_430	2008-04-08	12
-278	val_278	2008-04-08	12
-176	val_176	2008-04-08	12
-289	val_289	2008-04-08	12
-221	val_221	2008-04-08	12
-65	val_65	2008-04-08	12
-318	val_318	2008-04-08	12
-332	val_332	2008-04-08	12
-311	val_311	2008-04-08	12
-275	val_275	2008-04-08	12
-137	val_137	2008-04-08	12
-241	val_241	2008-04-08	12
-83	val_83	2008-04-08	12
-333	val_333	2008-04-08	12
-180	val_180	2008-04-08	12
-284	val_284	2008-04-08	12
-12	val_12	2008-04-08	12
-230	val_230	2008-04-08	12
-181	val_181	2008-04-08	12
-67	val_67	2008-04-08	12
-260	val_260	2008-04-08	12
-404	val_404	2008-04-08	12
-384	val_384	2008-04-08	12
-489	val_489	2008-04-08	12
-353	val_353	2008-04-08	12
-373	val_373	2008-04-08	12
-272	val_272	2008-04-08	12
-138	val_138	2008-04-08	12
-217	val_217	2008-04-08	12
-84	val_84	2008-04-08	12
-348	val_348	2008-04-08	12
-466	val_466	2008-04-08	12
-58	val_58	2008-04-08	12
-8	val_8	2008-04-08	12
-411	val_411	2008-04-08	12
-230	val_230	2008-04-08	12
-208	val_208	2008-04-08	12
-348	val_348	2008-04-08	12
-24	val_24	2008-04-08	12
-463	val_463	2008-04-08	12
-431	val_431	2008-04-08	12
-179	val_179	2008-04-08	12
-172	val_172	2008-04-08	12
-42	val_42	2008-04-08	12
-129	val_129	2008-04-08	12
-158	val_158	2008-04-08	12
-119	val_119	2008-04-08	12
-496	val_496	2008-04-08	12
-0	val_0	2008-04-08	12
-322	val_322	2008-04-08	12
-197	val_197	2008-04-08	12
-468	val_468	2008-04-08	12
-393	val_393	2008-04-08	12
-454	val_454	2008-04-08	12
-100	val_100	2008-04-08	12
-298	val_298	2008-04-08	12
-199	val_199	2008-04-08	12
-191	val_191	2008-04-08	12
-418	val_418	2008-04-08	12
-96	val_96	2008-04-08	12
-26	val_26	2008-04-08	12
-165	val_165	2008-04-08	12
-327	val_327	2008-04-08	12
-230	val_230	2008-04-08	12
-205	val_205	2008-04-08	12
-120	val_120	2008-04-08	12
-131	val_131	2008-04-08	12
-51	val_51	2008-04-08	12
-404	val_404	2008-04-08	12
-43	val_43	2008-04-08	12
-436	val_436	2008-04-08	12
-156	val_156	2008-04-08	12
-469	val_469	2008-04-08	12
-468	val_468	2008-04-08	12
-308	val_308	2008-04-08	12
-95	val_95	2008-04-08	12
-196	val_196	2008-04-08	12
-288	val_288	2008-04-08	12
-481	val_481	2008-04-08	12
-457	val_457	2008-04-08	12
-98	val_98	2008-04-08	12
-282	val_282	2008-04-08	12
-197	val_197	2008-04-08	12
-187	val_187	2008-04-08	12
-318	val_318	2008-04-08	12
-318	val_318	2008-04-08	12
-409	val_409	2008-04-08	12
-470	val_470	2008-04-08	12
-137	val_137	2008-04-08	12
-369	val_369	2008-04-08	12
-316	val_316	2008-04-08	12
-169	val_169	2008-04-08	12
-413	val_413	2008-04-08	12
-85	val_85	2008-04-08	12
-77	val_77	2008-04-08	12
-0	val_0	2008-04-08	12
-490	val_490	2008-04-08	12
-87	val_87	2008-04-08	12
-364	val_364	2008-04-08	12
-179	val_179	2008-04-08	12
-118	val_118	2008-04-08	12
-134	val_134	2008-04-08	12
-395	val_395	2008-04-08	12
-282	val_282	2008-04-08	12
-138	val_138	2008-04-08	12
-238	val_238	2008-04-08	12
-419	val_419	2008-04-08	12
-15	val_15	2008-04-08	12
-118	val_118	2008-04-08	12
-72	val_72	2008-04-08	12
-90	val_90	2008-04-08	12
-307	val_307	2008-04-08	12
-19	val_19	2008-04-08	12
-435	val_435	2008-04-08	12
-10	val_10	2008-04-08	12
-277	val_277	2008-04-08	12
-273	val_273	2008-04-08	12
-306	val_306	2008-04-08	12
-224	val_224	2008-04-08	12
-309	val_309	2008-04-08	12
-389	val_389	2008-04-08	12
-327	val_327	2008-04-08	12
-242	val_242	2008-04-08	12
-369	val_369	2008-04-08	12
-392	val_392	2008-04-08	12
-272	val_272	2008-04-08	12
-331	val_331	2008-04-08	12
-401	val_401	2008-04-08	12
-242	val_242	2008-04-08	12
-452	val_452	2008-04-08	12
-177	val_177	2008-04-08	12
-226	val_226	2008-04-08	12
-5	val_5	2008-04-08	12
-497	val_497	2008-04-08	12
-402	val_402	2008-04-08	12
-396	val_396	2008-04-08	12
-317	val_317	2008-04-08	12
-395	val_395	2008-04-08	12
-58	val_58	2008-04-08	12
-35	val_35	2008-04-08	12
-336	val_336	2008-04-08	12
-95	val_95	2008-04-08	12
-11	val_11	2008-04-08	12
-168	val_168	2008-04-08	12
-34	val_34	2008-04-08	12
-229	val_229	2008-04-08	12
-233	val_233	2008-04-08	12
-143	val_143	2008-04-08	12
-472	val_472	2008-04-08	12
-322	val_322	2008-04-08	12
-498	val_498	2008-04-08	12
-160	val_160	2008-04-08	12
-195	val_195	2008-04-08	12
-42	val_42	2008-04-08	12
-321	val_321	2008-04-08	12
-430	val_430	2008-04-08	12
-119	val_119	2008-04-08	12
-489	val_489	2008-04-08	12
-458	val_458	2008-04-08	12
-78	val_78	2008-04-08	12
-76	val_76	2008-04-08	12
-41	val_41	2008-04-08	12
-223	val_223	2008-04-08	12
-492	val_492	2008-04-08	12
-149	val_149	2008-04-08	12
-449	val_449	2008-04-08	12
-218	val_218	2008-04-08	12
-228	val_228	2008-04-08	12
-138	val_138	2008-04-08	12
-453	val_453	2008-04-08	12
-30	val_30	2008-04-08	12
-209	val_209	2008-04-08	12
-64	val_64	2008-04-08	12
-468	val_468	2008-04-08	12
-76	val_76	2008-04-08	12
-74	val_74	2008-04-08	12
-342	val_342	2008-04-08	12
-69	val_69	2008-04-08	12
-230	val_230	2008-04-08	12
-33	val_33	2008-04-08	12
-368	val_368	2008-04-08	12
-103	val_103	2008-04-08	12
-296	val_296	2008-04-08	12
-113	val_113	2008-04-08	12
-216	val_216	2008-04-08	12
-367	val_367	2008-04-08	12
-344	val_344	2008-04-08	12
-167	val_167	2008-04-08	12
-274	val_274	2008-04-08	12
-219	val_219	2008-04-08	12
-239	val_239	2008-04-08	12
-485	val_485	2008-04-08	12
-116	val_116	2008-04-08	12
-223	val_223	2008-04-08	12
-256	val_256	2008-04-08	12
-263	val_263	2008-04-08	12
-70	val_70	2008-04-08	12
-487	val_487	2008-04-08	12
-480	val_480	2008-04-08	12
-401	val_401	2008-04-08	12
-288	val_288	2008-04-08	12
-191	val_191	2008-04-08	12
-5	val_5	2008-04-08	12
-244	val_244	2008-04-08	12
-438	val_438	2008-04-08	12
-128	val_128	2008-04-08	12
-467	val_467	2008-04-08	12
-432	val_432	2008-04-08	12
-202	val_202	2008-04-08	12
-316	val_316	2008-04-08	12
-229	val_229	2008-04-08	12
-469	val_469	2008-04-08	12
-463	val_463	2008-04-08	12
-280	val_280	2008-04-08	12
-2	val_2	2008-04-08	12
-35	val_35	2008-04-08	12
-283	val_283	2008-04-08	12
-331	val_331	2008-04-08	12
-235	val_235	2008-04-08	12
-80	val_80	2008-04-08	12
-44	val_44	2008-04-08	12
-193	val_193	2008-04-08	12
-321	val_321	2008-04-08	12
-335	val_335	2008-04-08	12
-104	val_104	2008-04-08	12
-466	val_466	2008-04-08	12
-366	val_366	2008-04-08	12
-175	val_175	2008-04-08	12
-403	val_403	2008-04-08	12
-483	val_483	2008-04-08	12
-53	val_53	2008-04-08	12
-105	val_105	2008-04-08	12
-257	val_257	2008-04-08	12
-406	val_406	2008-04-08	12
-409	val_409	2008-04-08	12
-190	val_190	2008-04-08	12
-406	val_406	2008-04-08	12
-401	val_401	2008-04-08	12
-114	val_114	2008-04-08	12
-258	val_258	2008-04-08	12
-90	val_90	2008-04-08	12
-203	val_203	2008-04-08	12
-262	val_262	2008-04-08	12
-348	val_348	2008-04-08	12
-424	val_424	2008-04-08	12
-12	val_12	2008-04-08	12
-396	val_396	2008-04-08	12
-201	val_201	2008-04-08	12
-217	val_217	2008-04-08	12
-164	val_164	2008-04-08	12
-431	val_431	2008-04-08	12
-454	val_454	2008-04-08	12
-478	val_478	2008-04-08	12
-298	val_298	2008-04-08	12
-125	val_125	2008-04-08	12
-431	val_431	2008-04-08	12
-164	val_164	2008-04-08	12
-424	val_424	2008-04-08	12
-187	val_187	2008-04-08	12
-382	val_382	2008-04-08	12
-5	val_5	2008-04-08	12
-70	val_70	2008-04-08	12
-397	val_397	2008-04-08	12
-480	val_480	2008-04-08	12
-291	val_291	2008-04-08	12
-24	val_24	2008-04-08	12
-351	val_351	2008-04-08	12
-255	val_255	2008-04-08	12
-104	val_104	2008-04-08	12
-70	val_70	2008-04-08	12
-163	val_163	2008-04-08	12
-438	val_438	2008-04-08	12
-119	val_119	2008-04-08	12
-414	val_414	2008-04-08	12
-200	val_200	2008-04-08	12
-491	val_491	2008-04-08	12
-237	val_237	2008-04-08	12
-439	val_439	2008-04-08	12
-360	val_360	2008-04-08	12
-248	val_248	2008-04-08	12
-479	val_479	2008-04-08	12
-305	val_305	2008-04-08	12
-417	val_417	2008-04-08	12
-199	val_199	2008-04-08	12
-444	val_444	2008-04-08	12
-120	val_120	2008-04-08	12
-429	val_429	2008-04-08	12
-169	val_169	2008-04-08	12
-443	val_443	2008-04-08	12
-323	val_323	2008-04-08	12
-325	val_325	2008-04-08	12
-277	val_277	2008-04-08	12
-230	val_230	2008-04-08	12
-478	val_478	2008-04-08	12
-178	val_178	2008-04-08	12
-468	val_468	2008-04-08	12
-310	val_310	2008-04-08	12
-317	val_317	2008-04-08	12
-333	val_333	2008-04-08	12
-493	val_493	2008-04-08	12
-460	val_460	2008-04-08	12
-207	val_207	2008-04-08	12
-249	val_249	2008-04-08	12
-265	val_265	2008-04-08	12
-480	val_480	2008-04-08	12
-83	val_83	2008-04-08	12
-136	val_136	2008-04-08	12
-353	val_353	2008-04-08	12
-172	val_172	2008-04-08	12
-214	val_214	2008-04-08	12
-462	val_462	2008-04-08	12
-233	val_233	2008-04-08	12
-406	val_406	2008-04-08	12
-133	val_133	2008-04-08	12
-175	val_175	2008-04-08	12
-189	val_189	2008-04-08	12
-454	val_454	2008-04-08	12
-375	val_375	2008-04-08	12
-401	val_401	2008-04-08	12
-421	val_421	2008-04-08	12
-407	val_407	2008-04-08	12
-384	val_384	2008-04-08	12
-256	val_256	2008-04-08	12
-26	val_26	2008-04-08	12
-134	val_134	2008-04-08	12
-67	val_67	2008-04-08	12
-384	val_384	2008-04-08	12
-379	val_379	2008-04-08	12
-18	val_18	2008-04-08	12
-462	val_462	2008-04-08	12
-492	val_492	2008-04-08	12
-100	val_100	2008-04-08	12
-298	val_298	2008-04-08	12
-9	val_9	2008-04-08	12
-341	val_341	2008-04-08	12
-498	val_498	2008-04-08	12
-146	val_146	2008-04-08	12
-458	val_458	2008-04-08	12
-362	val_362	2008-04-08	12
-186	val_186	2008-04-08	12
-285	val_285	2008-04-08	12
-348	val_348	2008-04-08	12
-167	val_167	2008-04-08	12
-18	val_18	2008-04-08	12
-273	val_273	2008-04-08	12
-183	val_183	2008-04-08	12
-281	val_281	2008-04-08	12
-344	val_344	2008-04-08	12
-97	val_97	2008-04-08	12
-469	val_469	2008-04-08	12
-315	val_315	2008-04-08	12
-84	val_84	2008-04-08	12
-28	val_28	2008-04-08	12
-37	val_37	2008-04-08	12
-448	val_448	2008-04-08	12
-152	val_152	2008-04-08	12
-348	val_348	2008-04-08	12
-307	val_307	2008-04-08	12
-194	val_194	2008-04-08	12
-414	val_414	2008-04-08	12
-477	val_477	2008-04-08	12
-222	val_222	2008-04-08	12
-126	val_126	2008-04-08	12
-90	val_90	2008-04-08	12
-169	val_169	2008-04-08	12
-403	val_403	2008-04-08	12
-400	val_400	2008-04-08	12
-200	val_200	2008-04-08	12
-97	val_97	2008-04-08	12
-238	val_238	2008-04-09	11
-86	val_86	2008-04-09	11
-311	val_311	2008-04-09	11
-27	val_27	2008-04-09	11
-165	val_165	2008-04-09	11
-409	val_409	2008-04-09	11
-255	val_255	2008-04-09	11
-278	val_278	2008-04-09	11
-98	val_98	2008-04-09	11
-484	val_484	2008-04-09	11
-265	val_265	2008-04-09	11
-193	val_193	2008-04-09	11
-401	val_401	2008-04-09	11
-150	val_150	2008-04-09	11
-273	val_273	2008-04-09	11
-224	val_224	2008-04-09	11
-369	val_369	2008-04-09	11
-66	val_66	2008-04-09	11
-128	val_128	2008-04-09	11
-213	val_213	2008-04-09	11
-146	val_146	2008-04-09	11
-406	val_406	2008-04-09	11
-429	val_429	2008-04-09	11
-374	val_374	2008-04-09	11
-152	val_152	2008-04-09	11
-469	val_469	2008-04-09	11
-145	val_145	2008-04-09	11
-495	val_495	2008-04-09	11
-37	val_37	2008-04-09	11
-327	val_327	2008-04-09	11
-281	val_281	2008-04-09	11
-277	val_277	2008-04-09	11
-209	val_209	2008-04-09	11
-15	val_15	2008-04-09	11
-82	val_82	2008-04-09	11
-403	val_403	2008-04-09	11
-166	val_166	2008-04-09	11
-417	val_417	2008-04-09	11
-430	val_430	2008-04-09	11
-252	val_252	2008-04-09	11
-292	val_292	2008-04-09	11
-219	val_219	2008-04-09	11
-287	val_287	2008-04-09	11
-153	val_153	2008-04-09	11
-193	val_193	2008-04-09	11
-338	val_338	2008-04-09	11
-446	val_446	2008-04-09	11
-459	val_459	2008-04-09	11
-394	val_394	2008-04-09	11
-237	val_237	2008-04-09	11
-482	val_482	2008-04-09	11
-174	val_174	2008-04-09	11
-413	val_413	2008-04-09	11
-494	val_494	2008-04-09	11
-207	val_207	2008-04-09	11
-199	val_199	2008-04-09	11
-466	val_466	2008-04-09	11
-208	val_208	2008-04-09	11
-174	val_174	2008-04-09	11
-399	val_399	2008-04-09	11
-396	val_396	2008-04-09	11
-247	val_247	2008-04-09	11
-417	val_417	2008-04-09	11
-489	val_489	2008-04-09	11
-162	val_162	2008-04-09	11
-377	val_377	2008-04-09	11
-397	val_397	2008-04-09	11
-309	val_309	2008-04-09	11
-365	val_365	2008-04-09	11
-266	val_266	2008-04-09	11
-439	val_439	2008-04-09	11
-342	val_342	2008-04-09	11
-367	val_367	2008-04-09	11
-325	val_325	2008-04-09	11
-167	val_167	2008-04-09	11
-195	val_195	2008-04-09	11
-475	val_475	2008-04-09	11
-17	val_17	2008-04-09	11
-113	val_113	2008-04-09	11
-155	val_155	2008-04-09	11
-203	val_203	2008-04-09	11
-339	val_339	2008-04-09	11
-0	val_0	2008-04-09	11
-455	val_455	2008-04-09	11
-128	val_128	2008-04-09	11
-311	val_311	2008-04-09	11
-316	val_316	2008-04-09	11
-57	val_57	2008-04-09	11
-302	val_302	2008-04-09	11
-205	val_205	2008-04-09	11
-149	val_149	2008-04-09	11
-438	val_438	2008-04-09	11
-345	val_345	2008-04-09	11
-129	val_129	2008-04-09	11
-170	val_170	2008-04-09	11
-20	val_20	2008-04-09	11
-489	val_489	2008-04-09	11
-157	val_157	2008-04-09	11
-378	val_378	2008-04-09	11
-221	val_221	2008-04-09	11
-92	val_92	2008-04-09	11
-111	val_111	2008-04-09	11
-47	val_47	2008-04-09	11
-72	val_72	2008-04-09	11
-4	val_4	2008-04-09	11
-280	val_280	2008-04-09	11
-35	val_35	2008-04-09	11
-427	val_427	2008-04-09	11
-277	val_277	2008-04-09	11
-208	val_208	2008-04-09	11
-356	val_356	2008-04-09	11
-399	val_399	2008-04-09	11
-169	val_169	2008-04-09	11
-382	val_382	2008-04-09	11
-498	val_498	2008-04-09	11
-125	val_125	2008-04-09	11
-386	val_386	2008-04-09	11
-437	val_437	2008-04-09	11
-469	val_469	2008-04-09	11
-192	val_192	2008-04-09	11
-286	val_286	2008-04-09	11
-187	val_187	2008-04-09	11
-176	val_176	2008-04-09	11
-54	val_54	2008-04-09	11
-459	val_459	2008-04-09	11
-51	val_51	2008-04-09	11
-138	val_138	2008-04-09	11
-103	val_103	2008-04-09	11
-239	val_239	2008-04-09	11
-213	val_213	2008-04-09	11
-216	val_216	2008-04-09	11
-430	val_430	2008-04-09	11
-278	val_278	2008-04-09	11
-176	val_176	2008-04-09	11
-289	val_289	2008-04-09	11
-221	val_221	2008-04-09	11
-65	val_65	2008-04-09	11
-318	val_318	2008-04-09	11
-332	val_332	2008-04-09	11
-311	val_311	2008-04-09	11
-275	val_275	2008-04-09	11
-137	val_137	2008-04-09	11
-241	val_241	2008-04-09	11
-83	val_83	2008-04-09	11
-333	val_333	2008-04-09	11
-180	val_180	2008-04-09	11
-284	val_284	2008-04-09	11
-12	val_12	2008-04-09	11
-230	val_230	2008-04-09	11
-181	val_181	2008-04-09	11
-67	val_67	2008-04-09	11
-260	val_260	2008-04-09	11
-404	val_404	2008-04-09	11
-384	val_384	2008-04-09	11
-489	val_489	2008-04-09	11
-353	val_353	2008-04-09	11
-373	val_373	2008-04-09	11
-272	val_272	2008-04-09	11
-138	val_138	2008-04-09	11
-217	val_217	2008-04-09	11
-84	val_84	2008-04-09	11
-348	val_348	2008-04-09	11
-466	val_466	2008-04-09	11
-58	val_58	2008-04-09	11
-8	val_8	2008-04-09	11
-411	val_411	2008-04-09	11
-230	val_230	2008-04-09	11
-208	val_208	2008-04-09	11
-348	val_348	2008-04-09	11
-24	val_24	2008-04-09	11
-463	val_463	2008-04-09	11
-431	val_431	2008-04-09	11
-179	val_179	2008-04-09	11
-172	val_172	2008-04-09	11
-42	val_42	2008-04-09	11
-129	val_129	2008-04-09	11
-158	val_158	2008-04-09	11
-119	val_119	2008-04-09	11
-496	val_496	2008-04-09	11
-0	val_0	2008-04-09	11
-322	val_322	2008-04-09	11
-197	val_197	2008-04-09	11
-468	val_468	2008-04-09	11
-393	val_393	2008-04-09	11
-454	val_454	2008-04-09	11
-100	val_100	2008-04-09	11
-298	val_298	2008-04-09	11
-199	val_199	2008-04-09	11
-191	val_191	2008-04-09	11
-418	val_418	2008-04-09	11
-96	val_96	2008-04-09	11
-26	val_26	2008-04-09	11
-165	val_165	2008-04-09	11
-327	val_327	2008-04-09	11
-230	val_230	2008-04-09	11
-205	val_205	2008-04-09	11
-120	val_120	2008-04-09	11
-131	val_131	2008-04-09	11
-51	val_51	2008-04-09	11
-404	val_404	2008-04-09	11
-43	val_43	2008-04-09	11
-436	val_436	2008-04-09	11
-156	val_156	2008-04-09	11
-469	val_469	2008-04-09	11
-468	val_468	2008-04-09	11
-308	val_308	2008-04-09	11
-95	val_95	2008-04-09	11
-196	val_196	2008-04-09	11
-288	val_288	2008-04-09	11
-481	val_481	2008-04-09	11
-457	val_457	2008-04-09	11
-98	val_98	2008-04-09	11
-282	val_282	2008-04-09	11
-197	val_197	2008-04-09	11
-187	val_187	2008-04-09	11
-318	val_318	2008-04-09	11
-318	val_318	2008-04-09	11
-409	val_409	2008-04-09	11
-470	val_470	2008-04-09	11
-137	val_137	2008-04-09	11
-369	val_369	2008-04-09	11
-316	val_316	2008-04-09	11
-169	val_169	2008-04-09	11
-413	val_413	2008-04-09	11
-85	val_85	2008-04-09	11
-77	val_77	2008-04-09	11
-0	val_0	2008-04-09	11
-490	val_490	2008-04-09	11
-87	val_87	2008-04-09	11
-364	val_364	2008-04-09	11
-179	val_179	2008-04-09	11
-118	val_118	2008-04-09	11
-134	val_134	2008-04-09	11
-395	val_395	2008-04-09	11
-282	val_282	2008-04-09	11
-138	val_138	2008-04-09	11
-238	val_238	2008-04-09	11
-419	val_419	2008-04-09	11
-15	val_15	2008-04-09	11
-118	val_118	2008-04-09	11
-72	val_72	2008-04-09	11
-90	val_90	2008-04-09	11
-307	val_307	2008-04-09	11
-19	val_19	2008-04-09	11
-435	val_435	2008-04-09	11
-10	val_10	2008-04-09	11
-277	val_277	2008-04-09	11
-273	val_273	2008-04-09	11
-306	val_306	2008-04-09	11
-224	val_224	2008-04-09	11
-309	val_309	2008-04-09	11
-389	val_389	2008-04-09	11
-327	val_327	2008-04-09	11
-242	val_242	2008-04-09	11
-369	val_369	2008-04-09	11
-392	val_392	2008-04-09	11
-272	val_272	2008-04-09	11
-331	val_331	2008-04-09	11
-401	val_401	2008-04-09	11
-242	val_242	2008-04-09	11
-452	val_452	2008-04-09	11
-177	val_177	2008-04-09	11
-226	val_226	2008-04-09	11
-5	val_5	2008-04-09	11
-497	val_497	2008-04-09	11
-402	val_402	2008-04-09	11
-396	val_396	2008-04-09	11
-317	val_317	2008-04-09	11
-395	val_395	2008-04-09	11
-58	val_58	2008-04-09	11
-35	val_35	2008-04-09	11
-336	val_336	2008-04-09	11
-95	val_95	2008-04-09	11
-11	val_11	2008-04-09	11
-168	val_168	2008-04-09	11
-34	val_34	2008-04-09	11
-229	val_229	2008-04-09	11
-233	val_233	2008-04-09	11
-143	val_143	2008-04-09	11
-472	val_472	2008-04-09	11
-322	val_322	2008-04-09	11
-498	val_498	2008-04-09	11
-160	val_160	2008-04-09	11
-195	val_195	2008-04-09	11
-42	val_42	2008-04-09	11
-321	val_321	2008-04-09	11
-430	val_430	2008-04-09	11
-119	val_119	2008-04-09	11
-489	val_489	2008-04-09	11
-458	val_458	2008-04-09	11
-78	val_78	2008-04-09	11
-76	val_76	2008-04-09	11
-41	val_41	2008-04-09	11
-223	val_223	2008-04-09	11
-492	val_492	2008-04-09	11
-149	val_149	2008-04-09	11
-449	val_449	2008-04-09	11
-218	val_218	2008-04-09	11
-228	val_228	2008-04-09	11
-138	val_138	2008-04-09	11
-453	val_453	2008-04-09	11
-30	val_30	2008-04-09	11
-209	val_209	2008-04-09	11
-64	val_64	2008-04-09	11
-468	val_468	2008-04-09	11
-76	val_76	2008-04-09	11
-74	val_74	2008-04-09	11
-342	val_342	2008-04-09	11
-69	val_69	2008-04-09	11
-230	val_230	2008-04-09	11
-33	val_33	2008-04-09	11
-368	val_368	2008-04-09	11
-103	val_103	2008-04-09	11
-296	val_296	2008-04-09	11
-113	val_113	2008-04-09	11
-216	val_216	2008-04-09	11
-367	val_367	2008-04-09	11
-344	val_344	2008-04-09	11
-167	val_167	2008-04-09	11
-274	val_274	2008-04-09	11
-219	val_219	2008-04-09	11
-239	val_239	2008-04-09	11
-485	val_485	2008-04-09	11
-116	val_116	2008-04-09	11
-223	val_223	2008-04-09	11
-256	val_256	2008-04-09	11
-263	val_263	2008-04-09	11
-70	val_70	2008-04-09	11
-487	val_487	2008-04-09	11
-480	val_480	2008-04-09	11
-401	val_401	2008-04-09	11
-288	val_288	2008-04-09	11
-191	val_191	2008-04-09	11
-5	val_5	2008-04-09	11
-244	val_244	2008-04-09	11
-438	val_438	2008-04-09	11
-128	val_128	2008-04-09	11
-467	val_467	2008-04-09	11
-432	val_432	2008-04-09	11
-202	val_202	2008-04-09	11
-316	val_316	2008-04-09	11
-229	val_229	2008-04-09	11
-469	val_469	2008-04-09	11
-463	val_463	2008-04-09	11
-280	val_280	2008-04-09	11
-2	val_2	2008-04-09	11
-35	val_35	2008-04-09	11
-283	val_283	2008-04-09	11
-331	val_331	2008-04-09	11
-235	val_235	2008-04-09	11
-80	val_80	2008-04-09	11
-44	val_44	2008-04-09	11
-193	val_193	2008-04-09	11
-321	val_321	2008-04-09	11
-335	val_335	2008-04-09	11
-104	val_104	2008-04-09	11
-466	val_466	2008-04-09	11
-366	val_366	2008-04-09	11
-175	val_175	2008-04-09	11
-403	val_403	2008-04-09	11
-483	val_483	2008-04-09	11
-53	val_53	2008-04-09	11
-105	val_105	2008-04-09	11
-257	val_257	2008-04-09	11
-406	val_406	2008-04-09	11
-409	val_409	2008-04-09	11
-190	val_190	2008-04-09	11
-406	val_406	2008-04-09	11
-401	val_401	2008-04-09	11
-114	val_114	2008-04-09	11
-258	val_258	2008-04-09	11
-90	val_90	2008-04-09	11
-203	val_203	2008-04-09	11
-262	val_262	2008-04-09	11
-348	val_348	2008-04-09	11
-424	val_424	2008-04-09	11
-12	val_12	2008-04-09	11
-396	val_396	2008-04-09	11
-201	val_201	2008-04-09	11
-217	val_217	2008-04-09	11
-164	val_164	2008-04-09	11
-431	val_431	2008-04-09	11
-454	val_454	2008-04-09	11
-478	val_478	2008-04-09	11
-298	val_298	2008-04-09	11
-125	val_125	2008-04-09	11
-431	val_431	2008-04-09	11
-164	val_164	2008-04-09	11
-424	val_424	2008-04-09	11
-187	val_187	2008-04-09	11
-382	val_382	2008-04-09	11
-5	val_5	2008-04-09	11
-70	val_70	2008-04-09	11
-397	val_397	2008-04-09	11
-480	val_480	2008-04-09	11
-291	val_291	2008-04-09	11
-24	val_24	2008-04-09	11
-351	val_351	2008-04-09	11
-255	val_255	2008-04-09	11
-104	val_104	2008-04-09	11
-70	val_70	2008-04-09	11
-163	val_163	2008-04-09	11
-438	val_438	2008-04-09	11
-119	val_119	2008-04-09	11
-414	val_414	2008-04-09	11
-200	val_200	2008-04-09	11
-491	val_491	2008-04-09	11
-237	val_237	2008-04-09	11
-439	val_439	2008-04-09	11
-360	val_360	2008-04-09	11
-248	val_248	2008-04-09	11
-479	val_479	2008-04-09	11
-305	val_305	2008-04-09	11
-417	val_417	2008-04-09	11
-199	val_199	2008-04-09	11
-444	val_444	2008-04-09	11
-120	val_120	2008-04-09	11
-429	val_429	2008-04-09	11
-169	val_169	2008-04-09	11
-443	val_443	2008-04-09	11
-323	val_323	2008-04-09	11
-325	val_325	2008-04-09	11
-277	val_277	2008-04-09	11
-230	val_230	2008-04-09	11
-478	val_478	2008-04-09	11
-178	val_178	2008-04-09	11
-468	val_468	2008-04-09	11
-310	val_310	2008-04-09	11
-317	val_317	2008-04-09	11
-333	val_333	2008-04-09	11
-493	val_493	2008-04-09	11
-460	val_460	2008-04-09	11
-207	val_207	2008-04-09	11
-249	val_249	2008-04-09	11
-265	val_265	2008-04-09	11
-480	val_480	2008-04-09	11
-83	val_83	2008-04-09	11
-136	val_136	2008-04-09	11
-353	val_353	2008-04-09	11
-172	val_172	2008-04-09	11
-214	val_214	2008-04-09	11
-462	val_462	2008-04-09	11
-233	val_233	2008-04-09	11
-406	val_406	2008-04-09	11
-133	val_133	2008-04-09	11
-175	val_175	2008-04-09	11
-189	val_189	2008-04-09	11
-454	val_454	2008-04-09	11
-375	val_375	2008-04-09	11
-401	val_401	2008-04-09	11
-421	val_421	2008-04-09	11
-407	val_407	2008-04-09	11
-384	val_384	2008-04-09	11
-256	val_256	2008-04-09	11
-26	val_26	2008-04-09	11
-134	val_134	2008-04-09	11
-67	val_67	2008-04-09	11
-384	val_384	2008-04-09	11
-379	val_379	2008-04-09	11
-18	val_18	2008-04-09	11
-462	val_462	2008-04-09	11
-492	val_492	2008-04-09	11
-100	val_100	2008-04-09	11
-298	val_298	2008-04-09	11
-9	val_9	2008-04-09	11
-341	val_341	2008-04-09	11
-498	val_498	2008-04-09	11
-146	val_146	2008-04-09	11
-458	val_458	2008-04-09	11
-362	val_362	2008-04-09	11
-186	val_186	2008-04-09	11
-285	val_285	2008-04-09	11
-348	val_348	2008-04-09	11
-167	val_167	2008-04-09	11
-18	val_18	2008-04-09	11
-273	val_273	2008-04-09	11
-183	val_183	2008-04-09	11
-281	val_281	2008-04-09	11
-344	val_344	2008-04-09	11
-97	val_97	2008-04-09	11
-469	val_469	2008-04-09	11
-315	val_315	2008-04-09	11
-84	val_84	2008-04-09	11
-28	val_28	2008-04-09	11
-37	val_37	2008-04-09	11
-448	val_448	2008-04-09	11
-152	val_152	2008-04-09	11
-348	val_348	2008-04-09	11
-307	val_307	2008-04-09	11
-194	val_194	2008-04-09	11
-414	val_414	2008-04-09	11
-477	val_477	2008-04-09	11
-222	val_222	2008-04-09	11
-126	val_126	2008-04-09	11
-90	val_90	2008-04-09	11
-169	val_169	2008-04-09	11
-403	val_403	2008-04-09	11
-400	val_400	2008-04-09	11
-200	val_200	2008-04-09	11
-97	val_97	2008-04-09	11
-238	val_238	2008-04-09	12
-86	val_86	2008-04-09	12
-311	val_311	2008-04-09	12
-27	val_27	2008-04-09	12
-165	val_165	2008-04-09	12
-409	val_409	2008-04-09	12
-255	val_255	2008-04-09	12
-278	val_278	2008-04-09	12
-98	val_98	2008-04-09	12
-484	val_484	2008-04-09	12
-265	val_265	2008-04-09	12
-193	val_193	2008-04-09	12
-401	val_401	2008-04-09	12
-150	val_150	2008-04-09	12
-273	val_273	2008-04-09	12
-224	val_224	2008-04-09	12
-369	val_369	2008-04-09	12
-66	val_66	2008-04-09	12
-128	val_128	2008-04-09	12
-213	val_213	2008-04-09	12
-146	val_146	2008-04-09	12
-406	val_406	2008-04-09	12
-429	val_429	2008-04-09	12
-374	val_374	2008-04-09	12
-152	val_152	2008-04-09	12
-469	val_469	2008-04-09	12
-145	val_145	2008-04-09	12
-495	val_495	2008-04-09	12
-37	val_37	2008-04-09	12
-327	val_327	2008-04-09	12
-281	val_281	2008-04-09	12
-277	val_277	2008-04-09	12
-209	val_209	2008-04-09	12
-15	val_15	2008-04-09	12
-82	val_82	2008-04-09	12
-403	val_403	2008-04-09	12
-166	val_166	2008-04-09	12
-417	val_417	2008-04-09	12
-430	val_430	2008-04-09	12
-252	val_252	2008-04-09	12
-292	val_292	2008-04-09	12
-219	val_219	2008-04-09	12
-287	val_287	2008-04-09	12
-153	val_153	2008-04-09	12
-193	val_193	2008-04-09	12
-338	val_338	2008-04-09	12
-446	val_446	2008-04-09	12
-459	val_459	2008-04-09	12
-394	val_394	2008-04-09	12
-237	val_237	2008-04-09	12
-482	val_482	2008-04-09	12
-174	val_174	2008-04-09	12
-413	val_413	2008-04-09	12
-494	val_494	2008-04-09	12
-207	val_207	2008-04-09	12
-199	val_199	2008-04-09	12
-466	val_466	2008-04-09	12
-208	val_208	2008-04-09	12
-174	val_174	2008-04-09	12
-399	val_399	2008-04-09	12
-396	val_396	2008-04-09	12
-247	val_247	2008-04-09	12
-417	val_417	2008-04-09	12
-489	val_489	2008-04-09	12
-162	val_162	2008-04-09	12
-377	val_377	2008-04-09	12
-397	val_397	2008-04-09	12
-309	val_309	2008-04-09	12
-365	val_365	2008-04-09	12
-266	val_266	2008-04-09	12
-439	val_439	2008-04-09	12
-342	val_342	2008-04-09	12
-367	val_367	2008-04-09	12
-325	val_325	2008-04-09	12
-167	val_167	2008-04-09	12
-195	val_195	2008-04-09	12
-475	val_475	2008-04-09	12
-17	val_17	2008-04-09	12
-113	val_113	2008-04-09	12
-155	val_155	2008-04-09	12
-203	val_203	2008-04-09	12
-339	val_339	2008-04-09	12
-0	val_0	2008-04-09	12
-455	val_455	2008-04-09	12
-128	val_128	2008-04-09	12
-311	val_311	2008-04-09	12
-316	val_316	2008-04-09	12
-57	val_57	2008-04-09	12
-302	val_302	2008-04-09	12
-205	val_205	2008-04-09	12
-149	val_149	2008-04-09	12
-438	val_438	2008-04-09	12
-345	val_345	2008-04-09	12
-129	val_129	2008-04-09	12
-170	val_170	2008-04-09	12
-20	val_20	2008-04-09	12
-489	val_489	2008-04-09	12
-157	val_157	2008-04-09	12
-378	val_378	2008-04-09	12
-221	val_221	2008-04-09	12
-92	val_92	2008-04-09	12
-111	val_111	2008-04-09	12
-47	val_47	2008-04-09	12
-72	val_72	2008-04-09	12
-4	val_4	2008-04-09	12
-280	val_280	2008-04-09	12
-35	val_35	2008-04-09	12
-427	val_427	2008-04-09	12
-277	val_277	2008-04-09	12
-208	val_208	2008-04-09	12
-356	val_356	2008-04-09	12
-399	val_399	2008-04-09	12
-169	val_169	2008-04-09	12
-382	val_382	2008-04-09	12
-498	val_498	2008-04-09	12
-125	val_125	2008-04-09	12
-386	val_386	2008-04-09	12
-437	val_437	2008-04-09	12
-469	val_469	2008-04-09	12
-192	val_192	2008-04-09	12
-286	val_286	2008-04-09	12
-187	val_187	2008-04-09	12
-176	val_176	2008-04-09	12
-54	val_54	2008-04-09	12
-459	val_459	2008-04-09	12
-51	val_51	2008-04-09	12
-138	val_138	2008-04-09	12
-103	val_103	2008-04-09	12
-239	val_239	2008-04-09	12
-213	val_213	2008-04-09	12
-216	val_216	2008-04-09	12
-430	val_430	2008-04-09	12
-278	val_278	2008-04-09	12
-176	val_176	2008-04-09	12
-289	val_289	2008-04-09	12
-221	val_221	2008-04-09	12
-65	val_65	2008-04-09	12
-318	val_318	2008-04-09	12
-332	val_332	2008-04-09	12
-311	val_311	2008-04-09	12
-275	val_275	2008-04-09	12
-137	val_137	2008-04-09	12
-241	val_241	2008-04-09	12
-83	val_83	2008-04-09	12
-333	val_333	2008-04-09	12
-180	val_180	2008-04-09	12
-284	val_284	2008-04-09	12
-12	val_12	2008-04-09	12
-230	val_230	2008-04-09	12
-181	val_181	2008-04-09	12
-67	val_67	2008-04-09	12
-260	val_260	2008-04-09	12
-404	val_404	2008-04-09	12
-384	val_384	2008-04-09	12
-489	val_489	2008-04-09	12
-353	val_353	2008-04-09	12
-373	val_373	2008-04-09	12
-272	val_272	2008-04-09	12
-138	val_138	2008-04-09	12
-217	val_217	2008-04-09	12
-84	val_84	2008-04-09	12
-348	val_348	2008-04-09	12
-466	val_466	2008-04-09	12
-58	val_58	2008-04-09	12
-8	val_8	2008-04-09	12
-411	val_411	2008-04-09	12
-230	val_230	2008-04-09	12
-208	val_208	2008-04-09	12
-348	val_348	2008-04-09	12
-24	val_24	2008-04-09	12
-463	val_463	2008-04-09	12
-431	val_431	2008-04-09	12
-179	val_179	2008-04-09	12
-172	val_172	2008-04-09	12
-42	val_42	2008-04-09	12
-129	val_129	2008-04-09	12
-158	val_158	2008-04-09	12
-119	val_119	2008-04-09	12
-496	val_496	2008-04-09	12
-0	val_0	2008-04-09	12
-322	val_322	2008-04-09	12
-197	val_197	2008-04-09	12
-468	val_468	2008-04-09	12
-393	val_393	2008-04-09	12
-454	val_454	2008-04-09	12
-100	val_100	2008-04-09	12
-298	val_298	2008-04-09	12
-199	val_199	2008-04-09	12
-191	val_191	2008-04-09	12
-418	val_418	2008-04-09	12
-96	val_96	2008-04-09	12
-26	val_26	2008-04-09	12
-165	val_165	2008-04-09	12
-327	val_327	2008-04-09	12
-230	val_230	2008-04-09	12
-205	val_205	2008-04-09	12
-120	val_120	2008-04-09	12
-131	val_131	2008-04-09	12
-51	val_51	2008-04-09	12
-404	val_404	2008-04-09	12
-43	val_43	2008-04-09	12
-436	val_436	2008-04-09	12
-156	val_156	2008-04-09	12
-469	val_469	2008-04-09	12
-468	val_468	2008-04-09	12
-308	val_308	2008-04-09	12
-95	val_95	2008-04-09	12
-196	val_196	2008-04-09	12
-288	val_288	2008-04-09	12
-481	val_481	2008-04-09	12
-457	val_457	2008-04-09	12
-98	val_98	2008-04-09	12
-282	val_282	2008-04-09	12
-197	val_197	2008-04-09	12
-187	val_187	2008-04-09	12
-318	val_318	2008-04-09	12
-318	val_318	2008-04-09	12
-409	val_409	2008-04-09	12
-470	val_470	2008-04-09	12
-137	val_137	2008-04-09	12
-369	val_369	2008-04-09	12
-316	val_316	2008-04-09	12
-169	val_169	2008-04-09	12
-413	val_413	2008-04-09	12
-85	val_85	2008-04-09	12
-77	val_77	2008-04-09	12
-0	val_0	2008-04-09	12
-490	val_490	2008-04-09	12
-87	val_87	2008-04-09	12
-364	val_364	2008-04-09	12
-179	val_179	2008-04-09	12
-118	val_118	2008-04-09	12
-134	val_134	2008-04-09	12
-395	val_395	2008-04-09	12
-282	val_282	2008-04-09	12
-138	val_138	2008-04-09	12
-238	val_238	2008-04-09	12
-419	val_419	2008-04-09	12
-15	val_15	2008-04-09	12
-118	val_118	2008-04-09	12
-72	val_72	2008-04-09	12
-90	val_90	2008-04-09	12
-307	val_307	2008-04-09	12
-19	val_19	2008-04-09	12
-435	val_435	2008-04-09	12
-10	val_10	2008-04-09	12
-277	val_277	2008-04-09	12
-273	val_273	2008-04-09	12
-306	val_306	2008-04-09	12
-224	val_224	2008-04-09	12
-309	val_309	2008-04-09	12
-389	val_389	2008-04-09	12
-327	val_327	2008-04-09	12
-242	val_242	2008-04-09	12
-369	val_369	2008-04-09	12
-392	val_392	2008-04-09	12
-272	val_272	2008-04-09	12
-331	val_331	2008-04-09	12
-401	val_401	2008-04-09	12
-242	val_242	2008-04-09	12
-452	val_452	2008-04-09	12
-177	val_177	2008-04-09	12
-226	val_226	2008-04-09	12
-5	val_5	2008-04-09	12
-497	val_497	2008-04-09	12
-402	val_402	2008-04-09	12
-396	val_396	2008-04-09	12
-317	val_317	2008-04-09	12
-395	val_395	2008-04-09	12
-58	val_58	2008-04-09	12
-35	val_35	2008-04-09	12
-336	val_336	2008-04-09	12
-95	val_95	2008-04-09	12
-11	val_11	2008-04-09	12
-168	val_168	2008-04-09	12
-34	val_34	2008-04-09	12
-229	val_229	2008-04-09	12
-233	val_233	2008-04-09	12
-143	val_143	2008-04-09	12
-472	val_472	2008-04-09	12
-322	val_322	2008-04-09	12
-498	val_498	2008-04-09	12
-160	val_160	2008-04-09	12
-195	val_195	2008-04-09	12
-42	val_42	2008-04-09	12
-321	val_321	2008-04-09	12
-430	val_430	2008-04-09	12
-119	val_119	2008-04-09	12
-489	val_489	2008-04-09	12
-458	val_458	2008-04-09	12
-78	val_78	2008-04-09	12
-76	val_76	2008-04-09	12
-41	val_41	2008-04-09	12
-223	val_223	2008-04-09	12
-492	val_492	2008-04-09	12
-149	val_149	2008-04-09	12
-449	val_449	2008-04-09	12
-218	val_218	2008-04-09	12
-228	val_228	2008-04-09	12
-138	val_138	2008-04-09	12
-453	val_453	2008-04-09	12
-30	val_30	2008-04-09	12
-209	val_209	2008-04-09	12
-64	val_64	2008-04-09	12
-468	val_468	2008-04-09	12
-76	val_76	2008-04-09	12
-74	val_74	2008-04-09	12
-342	val_342	2008-04-09	12
-69	val_69	2008-04-09	12
-230	val_230	2008-04-09	12
-33	val_33	2008-04-09	12
-368	val_368	2008-04-09	12
-103	val_103	2008-04-09	12
-296	val_296	2008-04-09	12
-113	val_113	2008-04-09	12
-216	val_216	2008-04-09	12
-367	val_367	2008-04-09	12
-344	val_344	2008-04-09	12
-167	val_167	2008-04-09	12
-274	val_274	2008-04-09	12
-219	val_219	2008-04-09	12
-239	val_239	2008-04-09	12
-485	val_485	2008-04-09	12
-116	val_116	2008-04-09	12
-223	val_223	2008-04-09	12
-256	val_256	2008-04-09	12
-263	val_263	2008-04-09	12
-70	val_70	2008-04-09	12
-487	val_487	2008-04-09	12
-480	val_480	2008-04-09	12
-401	val_401	2008-04-09	12
-288	val_288	2008-04-09	12
-191	val_191	2008-04-09	12
-5	val_5	2008-04-09	12
-244	val_244	2008-04-09	12
-438	val_438	2008-04-09	12
-128	val_128	2008-04-09	12
-467	val_467	2008-04-09	12
-432	val_432	2008-04-09	12
-202	val_202	2008-04-09	12
-316	val_316	2008-04-09	12
-229	val_229	2008-04-09	12
-469	val_469	2008-04-09	12
-463	val_463	2008-04-09	12
-280	val_280	2008-04-09	12
-2	val_2	2008-04-09	12
-35	val_35	2008-04-09	12
-283	val_283	2008-04-09	12
-331	val_331	2008-04-09	12
-235	val_235	2008-04-09	12
-80	val_80	2008-04-09	12
-44	val_44	2008-04-09	12
-193	val_193	2008-04-09	12
-321	val_321	2008-04-09	12
-335	val_335	2008-04-09	12
-104	val_104	2008-04-09	12
-466	val_466	2008-04-09	12
-366	val_366	2008-04-09	12
-175	val_175	2008-04-09	12
-403	val_403	2008-04-09	12
-483	val_483	2008-04-09	12
-53	val_53	2008-04-09	12
-105	val_105	2008-04-09	12
-257	val_257	2008-04-09	12
-406	val_406	2008-04-09	12
-409	val_409	2008-04-09	12
-190	val_190	2008-04-09	12
-406	val_406	2008-04-09	12
-401	val_401	2008-04-09	12
-114	val_114	2008-04-09	12
-258	val_258	2008-04-09	12
-90	val_90	2008-04-09	12
-203	val_203	2008-04-09	12
-262	val_262	2008-04-09	12
-348	val_348	2008-04-09	12
-424	val_424	2008-04-09	12
-12	val_12	2008-04-09	12
-396	val_396	2008-04-09	12
-201	val_201	2008-04-09	12
-217	val_217	2008-04-09	12
-164	val_164	2008-04-09	12
-431	val_431	2008-04-09	12
-454	val_454	2008-04-09	12
-478	val_478	2008-04-09	12
-298	val_298	2008-04-09	12
-125	val_125	2008-04-09	12
-431	val_431	2008-04-09	12
-164	val_164	2008-04-09	12
-424	val_424	2008-04-09	12
-187	val_187	2008-04-09	12
-382	val_382	2008-04-09	12
-5	val_5	2008-04-09	12
-70	val_70	2008-04-09	12
-397	val_397	2008-04-09	12
-480	val_480	2008-04-09	12
-291	val_291	2008-04-09	12
-24	val_24	2008-04-09	12
-351	val_351	2008-04-09	12
-255	val_255	2008-04-09	12
-104	val_104	2008-04-09	12
-70	val_70	2008-04-09	12
-163	val_163	2008-04-09	12
-438	val_438	2008-04-09	12
-119	val_119	2008-04-09	12
-414	val_414	2008-04-09	12
-200	val_200	2008-04-09	12
-491	val_491	2008-04-09	12
-237	val_237	2008-04-09	12
-439	val_439	2008-04-09	12
-360	val_360	2008-04-09	12
-248	val_248	2008-04-09	12
-479	val_479	2008-04-09	12
-305	val_305	2008-04-09	12
-417	val_417	2008-04-09	12
-199	val_199	2008-04-09	12
-444	val_444	2008-04-09	12
-120	val_120	2008-04-09	12
-429	val_429	2008-04-09	12
-169	val_169	2008-04-09	12
-443	val_443	2008-04-09	12
-323	val_323	2008-04-09	12
-325	val_325	2008-04-09	12
-277	val_277	2008-04-09	12
-230	val_230	2008-04-09	12
-478	val_478	2008-04-09	12
-178	val_178	2008-04-09	12
-468	val_468	2008-04-09	12
-310	val_310	2008-04-09	12
-317	val_317	2008-04-09	12
-333	val_333	2008-04-09	12
-493	val_493	2008-04-09	12
-460	val_460	2008-04-09	12
-207	val_207	2008-04-09	12
-249	val_249	2008-04-09	12
-265	val_265	2008-04-09	12
-480	val_480	2008-04-09	12
-83	val_83	2008-04-09	12
-136	val_136	2008-04-09	12
-353	val_353	2008-04-09	12
-172	val_172	2008-04-09	12
-214	val_214	2008-04-09	12
-462	val_462	2008-04-09	12
-233	val_233	2008-04-09	12
-406	val_406	2008-04-09	12
-133	val_133	2008-04-09	12
-175	val_175	2008-04-09	12
-189	val_189	2008-04-09	12
-454	val_454	2008-04-09	12
-375	val_375	2008-04-09	12
-401	val_401	2008-04-09	12
-421	val_421	2008-04-09	12
-407	val_407	2008-04-09	12
-384	val_384	2008-04-09	12
-256	val_256	2008-04-09	12
-26	val_26	2008-04-09	12
-134	val_134	2008-04-09	12
-67	val_67	2008-04-09	12
-384	val_384	2008-04-09	12
-379	val_379	2008-04-09	12
-18	val_18	2008-04-09	12
-462	val_462	2008-04-09	12
-492	val_492	2008-04-09	12
-100	val_100	2008-04-09	12
-298	val_298	2008-04-09	12
-9	val_9	2008-04-09	12
-341	val_341	2008-04-09	12
-498	val_498	2008-04-09	12
-146	val_146	2008-04-09	12
-458	val_458	2008-04-09	12
-362	val_362	2008-04-09	12
-186	val_186	2008-04-09	12
-285	val_285	2008-04-09	12
-348	val_348	2008-04-09	12
-167	val_167	2008-04-09	12
-18	val_18	2008-04-09	12
-273	val_273	2008-04-09	12
-183	val_183	2008-04-09	12
-281	val_281	2008-04-09	12
-344	val_344	2008-04-09	12
-97	val_97	2008-04-09	12
-469	val_469	2008-04-09	12
-315	val_315	2008-04-09	12
-84	val_84	2008-04-09	12
-28	val_28	2008-04-09	12
-37	val_37	2008-04-09	12
-448	val_448	2008-04-09	12
-152	val_152	2008-04-09	12
-348	val_348	2008-04-09	12
-307	val_307	2008-04-09	12
-194	val_194	2008-04-09	12
-414	val_414	2008-04-09	12
-477	val_477	2008-04-09	12
-222	val_222	2008-04-09	12
-126	val_126	2008-04-09	12
-90	val_90	2008-04-09	12
-169	val_169	2008-04-09	12
-403	val_403	2008-04-09	12
-400	val_400	2008-04-09	12
-200	val_200	2008-04-09	12
-97	val_97	2008-04-09	12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part4-10-297cf42ec45b5aa78b80d9c35a79555a b/sql/hive/src/test/resources/golden/load_dyn_part4-10-297cf42ec45b5aa78b80d9c35a79555a
new file mode 100644
index 0000000000000000000000000000000000000000..699bddc10b2ef18e015e51103f8341dfac2e7d19
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/load_dyn_part4-10-297cf42ec45b5aa78b80d9c35a79555a
@@ -0,0 +1,1500 @@
+0	val_0	2008-04-08	11
+0	val_0	2008-04-08	11
+0	val_0	2008-04-08	11
+2	val_2	2008-04-08	11
+4	val_4	2008-04-08	11
+5	val_5	2008-04-08	11
+5	val_5	2008-04-08	11
+5	val_5	2008-04-08	11
+8	val_8	2008-04-08	11
+9	val_9	2008-04-08	11
+10	val_10	2008-04-08	11
+11	val_11	2008-04-08	11
+12	val_12	2008-04-08	11
+12	val_12	2008-04-08	11
+15	val_15	2008-04-08	11
+15	val_15	2008-04-08	11
+17	val_17	2008-04-08	11
+18	val_18	2008-04-08	11
+18	val_18	2008-04-08	11
+19	val_19	2008-04-08	11
+20	val_20	2008-04-08	11
+24	val_24	2008-04-08	11
+24	val_24	2008-04-08	11
+26	val_26	2008-04-08	11
+26	val_26	2008-04-08	11
+27	val_27	2008-04-08	11
+28	val_28	2008-04-08	11
+30	val_30	2008-04-08	11
+33	val_33	2008-04-08	11
+34	val_34	2008-04-08	11
+35	val_35	2008-04-08	11
+35	val_35	2008-04-08	11
+35	val_35	2008-04-08	11
+37	val_37	2008-04-08	11
+37	val_37	2008-04-08	11
+41	val_41	2008-04-08	11
+42	val_42	2008-04-08	11
+42	val_42	2008-04-08	11
+43	val_43	2008-04-08	11
+44	val_44	2008-04-08	11
+47	val_47	2008-04-08	11
+51	val_51	2008-04-08	11
+51	val_51	2008-04-08	11
+53	val_53	2008-04-08	11
+54	val_54	2008-04-08	11
+57	val_57	2008-04-08	11
+58	val_58	2008-04-08	11
+58	val_58	2008-04-08	11
+64	val_64	2008-04-08	11
+65	val_65	2008-04-08	11
+66	val_66	2008-04-08	11
+67	val_67	2008-04-08	11
+67	val_67	2008-04-08	11
+69	val_69	2008-04-08	11
+70	val_70	2008-04-08	11
+70	val_70	2008-04-08	11
+70	val_70	2008-04-08	11
+72	val_72	2008-04-08	11
+72	val_72	2008-04-08	11
+74	val_74	2008-04-08	11
+76	val_76	2008-04-08	11
+76	val_76	2008-04-08	11
+77	val_77	2008-04-08	11
+78	val_78	2008-04-08	11
+80	val_80	2008-04-08	11
+82	val_82	2008-04-08	11
+83	val_83	2008-04-08	11
+83	val_83	2008-04-08	11
+84	val_84	2008-04-08	11
+84	val_84	2008-04-08	11
+85	val_85	2008-04-08	11
+86	val_86	2008-04-08	11
+87	val_87	2008-04-08	11
+90	val_90	2008-04-08	11
+90	val_90	2008-04-08	11
+90	val_90	2008-04-08	11
+92	val_92	2008-04-08	11
+95	val_95	2008-04-08	11
+95	val_95	2008-04-08	11
+96	val_96	2008-04-08	11
+97	val_97	2008-04-08	11
+97	val_97	2008-04-08	11
+98	val_98	2008-04-08	11
+98	val_98	2008-04-08	11
+100	val_100	2008-04-08	11
+100	val_100	2008-04-08	11
+103	val_103	2008-04-08	11
+103	val_103	2008-04-08	11
+104	val_104	2008-04-08	11
+104	val_104	2008-04-08	11
+105	val_105	2008-04-08	11
+111	val_111	2008-04-08	11
+113	val_113	2008-04-08	11
+113	val_113	2008-04-08	11
+114	val_114	2008-04-08	11
+116	val_116	2008-04-08	11
+118	val_118	2008-04-08	11
+118	val_118	2008-04-08	11
+119	val_119	2008-04-08	11
+119	val_119	2008-04-08	11
+119	val_119	2008-04-08	11
+120	val_120	2008-04-08	11
+120	val_120	2008-04-08	11
+125	val_125	2008-04-08	11
+125	val_125	2008-04-08	11
+126	val_126	2008-04-08	11
+128	val_128	2008-04-08	11
+128	val_128	2008-04-08	11
+128	val_128	2008-04-08	11
+129	val_129	2008-04-08	11
+129	val_129	2008-04-08	11
+131	val_131	2008-04-08	11
+133	val_133	2008-04-08	11
+134	val_134	2008-04-08	11
+134	val_134	2008-04-08	11
+136	val_136	2008-04-08	11
+137	val_137	2008-04-08	11
+137	val_137	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+143	val_143	2008-04-08	11
+145	val_145	2008-04-08	11
+146	val_146	2008-04-08	11
+146	val_146	2008-04-08	11
+149	val_149	2008-04-08	11
+149	val_149	2008-04-08	11
+150	val_150	2008-04-08	11
+152	val_152	2008-04-08	11
+152	val_152	2008-04-08	11
+153	val_153	2008-04-08	11
+155	val_155	2008-04-08	11
+156	val_156	2008-04-08	11
+157	val_157	2008-04-08	11
+158	val_158	2008-04-08	11
+160	val_160	2008-04-08	11
+162	val_162	2008-04-08	11
+163	val_163	2008-04-08	11
+164	val_164	2008-04-08	11
+164	val_164	2008-04-08	11
+165	val_165	2008-04-08	11
+165	val_165	2008-04-08	11
+166	val_166	2008-04-08	11
+167	val_167	2008-04-08	11
+167	val_167	2008-04-08	11
+167	val_167	2008-04-08	11
+168	val_168	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+170	val_170	2008-04-08	11
+172	val_172	2008-04-08	11
+172	val_172	2008-04-08	11
+174	val_174	2008-04-08	11
+174	val_174	2008-04-08	11
+175	val_175	2008-04-08	11
+175	val_175	2008-04-08	11
+176	val_176	2008-04-08	11
+176	val_176	2008-04-08	11
+177	val_177	2008-04-08	11
+178	val_178	2008-04-08	11
+179	val_179	2008-04-08	11
+179	val_179	2008-04-08	11
+180	val_180	2008-04-08	11
+181	val_181	2008-04-08	11
+183	val_183	2008-04-08	11
+186	val_186	2008-04-08	11
+187	val_187	2008-04-08	11
+187	val_187	2008-04-08	11
+187	val_187	2008-04-08	11
+189	val_189	2008-04-08	11
+190	val_190	2008-04-08	11
+191	val_191	2008-04-08	11
+191	val_191	2008-04-08	11
+192	val_192	2008-04-08	11
+193	val_193	2008-04-08	11
+193	val_193	2008-04-08	11
+193	val_193	2008-04-08	11
+194	val_194	2008-04-08	11
+195	val_195	2008-04-08	11
+195	val_195	2008-04-08	11
+196	val_196	2008-04-08	11
+197	val_197	2008-04-08	11
+197	val_197	2008-04-08	11
+199	val_199	2008-04-08	11
+199	val_199	2008-04-08	11
+199	val_199	2008-04-08	11
+200	val_200	2008-04-08	11
+200	val_200	2008-04-08	11
+201	val_201	2008-04-08	11
+202	val_202	2008-04-08	11
+203	val_203	2008-04-08	11
+203	val_203	2008-04-08	11
+205	val_205	2008-04-08	11
+205	val_205	2008-04-08	11
+207	val_207	2008-04-08	11
+207	val_207	2008-04-08	11
+208	val_208	2008-04-08	11
+208	val_208	2008-04-08	11
+208	val_208	2008-04-08	11
+209	val_209	2008-04-08	11
+209	val_209	2008-04-08	11
+213	val_213	2008-04-08	11
+213	val_213	2008-04-08	11
+214	val_214	2008-04-08	11
+216	val_216	2008-04-08	11
+216	val_216	2008-04-08	11
+217	val_217	2008-04-08	11
+217	val_217	2008-04-08	11
+218	val_218	2008-04-08	11
+219	val_219	2008-04-08	11
+219	val_219	2008-04-08	11
+221	val_221	2008-04-08	11
+221	val_221	2008-04-08	11
+222	val_222	2008-04-08	11
+223	val_223	2008-04-08	11
+223	val_223	2008-04-08	11
+224	val_224	2008-04-08	11
+224	val_224	2008-04-08	11
+226	val_226	2008-04-08	11
+228	val_228	2008-04-08	11
+229	val_229	2008-04-08	11
+229	val_229	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+233	val_233	2008-04-08	11
+233	val_233	2008-04-08	11
+235	val_235	2008-04-08	11
+237	val_237	2008-04-08	11
+237	val_237	2008-04-08	11
+238	val_238	2008-04-08	11
+238	val_238	2008-04-08	11
+239	val_239	2008-04-08	11
+239	val_239	2008-04-08	11
+241	val_241	2008-04-08	11
+242	val_242	2008-04-08	11
+242	val_242	2008-04-08	11
+244	val_244	2008-04-08	11
+247	val_247	2008-04-08	11
+248	val_248	2008-04-08	11
+249	val_249	2008-04-08	11
+252	val_252	2008-04-08	11
+255	val_255	2008-04-08	11
+255	val_255	2008-04-08	11
+256	val_256	2008-04-08	11
+256	val_256	2008-04-08	11
+257	val_257	2008-04-08	11
+258	val_258	2008-04-08	11
+260	val_260	2008-04-08	11
+262	val_262	2008-04-08	11
+263	val_263	2008-04-08	11
+265	val_265	2008-04-08	11
+265	val_265	2008-04-08	11
+266	val_266	2008-04-08	11
+272	val_272	2008-04-08	11
+272	val_272	2008-04-08	11
+273	val_273	2008-04-08	11
+273	val_273	2008-04-08	11
+273	val_273	2008-04-08	11
+274	val_274	2008-04-08	11
+275	val_275	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+278	val_278	2008-04-08	11
+278	val_278	2008-04-08	11
+280	val_280	2008-04-08	11
+280	val_280	2008-04-08	11
+281	val_281	2008-04-08	11
+281	val_281	2008-04-08	11
+282	val_282	2008-04-08	11
+282	val_282	2008-04-08	11
+283	val_283	2008-04-08	11
+284	val_284	2008-04-08	11
+285	val_285	2008-04-08	11
+286	val_286	2008-04-08	11
+287	val_287	2008-04-08	11
+288	val_288	2008-04-08	11
+288	val_288	2008-04-08	11
+289	val_289	2008-04-08	11
+291	val_291	2008-04-08	11
+292	val_292	2008-04-08	11
+296	val_296	2008-04-08	11
+298	val_298	2008-04-08	11
+298	val_298	2008-04-08	11
+298	val_298	2008-04-08	11
+302	val_302	2008-04-08	11
+305	val_305	2008-04-08	11
+306	val_306	2008-04-08	11
+307	val_307	2008-04-08	11
+307	val_307	2008-04-08	11
+308	val_308	2008-04-08	11
+309	val_309	2008-04-08	11
+309	val_309	2008-04-08	11
+310	val_310	2008-04-08	11
+311	val_311	2008-04-08	11
+311	val_311	2008-04-08	11
+311	val_311	2008-04-08	11
+315	val_315	2008-04-08	11
+316	val_316	2008-04-08	11
+316	val_316	2008-04-08	11
+316	val_316	2008-04-08	11
+317	val_317	2008-04-08	11
+317	val_317	2008-04-08	11
+318	val_318	2008-04-08	11
+318	val_318	2008-04-08	11
+318	val_318	2008-04-08	11
+321	val_321	2008-04-08	11
+321	val_321	2008-04-08	11
+322	val_322	2008-04-08	11
+322	val_322	2008-04-08	11
+323	val_323	2008-04-08	11
+325	val_325	2008-04-08	11
+325	val_325	2008-04-08	11
+327	val_327	2008-04-08	11
+327	val_327	2008-04-08	11
+327	val_327	2008-04-08	11
+331	val_331	2008-04-08	11
+331	val_331	2008-04-08	11
+332	val_332	2008-04-08	11
+333	val_333	2008-04-08	11
+333	val_333	2008-04-08	11
+335	val_335	2008-04-08	11
+336	val_336	2008-04-08	11
+338	val_338	2008-04-08	11
+339	val_339	2008-04-08	11
+341	val_341	2008-04-08	11
+342	val_342	2008-04-08	11
+342	val_342	2008-04-08	11
+344	val_344	2008-04-08	11
+344	val_344	2008-04-08	11
+345	val_345	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+351	val_351	2008-04-08	11
+353	val_353	2008-04-08	11
+353	val_353	2008-04-08	11
+356	val_356	2008-04-08	11
+360	val_360	2008-04-08	11
+362	val_362	2008-04-08	11
+364	val_364	2008-04-08	11
+365	val_365	2008-04-08	11
+366	val_366	2008-04-08	11
+367	val_367	2008-04-08	11
+367	val_367	2008-04-08	11
+368	val_368	2008-04-08	11
+369	val_369	2008-04-08	11
+369	val_369	2008-04-08	11
+369	val_369	2008-04-08	11
+373	val_373	2008-04-08	11
+374	val_374	2008-04-08	11
+375	val_375	2008-04-08	11
+377	val_377	2008-04-08	11
+378	val_378	2008-04-08	11
+379	val_379	2008-04-08	11
+382	val_382	2008-04-08	11
+382	val_382	2008-04-08	11
+384	val_384	2008-04-08	11
+384	val_384	2008-04-08	11
+384	val_384	2008-04-08	11
+386	val_386	2008-04-08	11
+389	val_389	2008-04-08	11
+392	val_392	2008-04-08	11
+393	val_393	2008-04-08	11
+394	val_394	2008-04-08	11
+395	val_395	2008-04-08	11
+395	val_395	2008-04-08	11
+396	val_396	2008-04-08	11
+396	val_396	2008-04-08	11
+396	val_396	2008-04-08	11
+397	val_397	2008-04-08	11
+397	val_397	2008-04-08	11
+399	val_399	2008-04-08	11
+399	val_399	2008-04-08	11
+400	val_400	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+402	val_402	2008-04-08	11
+403	val_403	2008-04-08	11
+403	val_403	2008-04-08	11
+403	val_403	2008-04-08	11
+404	val_404	2008-04-08	11
+404	val_404	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+407	val_407	2008-04-08	11
+409	val_409	2008-04-08	11
+409	val_409	2008-04-08	11
+409	val_409	2008-04-08	11
+411	val_411	2008-04-08	11
+413	val_413	2008-04-08	11
+413	val_413	2008-04-08	11
+414	val_414	2008-04-08	11
+414	val_414	2008-04-08	11
+417	val_417	2008-04-08	11
+417	val_417	2008-04-08	11
+417	val_417	2008-04-08	11
+418	val_418	2008-04-08	11
+419	val_419	2008-04-08	11
+421	val_421	2008-04-08	11
+424	val_424	2008-04-08	11
+424	val_424	2008-04-08	11
+427	val_427	2008-04-08	11
+429	val_429	2008-04-08	11
+429	val_429	2008-04-08	11
+430	val_430	2008-04-08	11
+430	val_430	2008-04-08	11
+430	val_430	2008-04-08	11
+431	val_431	2008-04-08	11
+431	val_431	2008-04-08	11
+431	val_431	2008-04-08	11
+432	val_432	2008-04-08	11
+435	val_435	2008-04-08	11
+436	val_436	2008-04-08	11
+437	val_437	2008-04-08	11
+438	val_438	2008-04-08	11
+438	val_438	2008-04-08	11
+438	val_438	2008-04-08	11
+439	val_439	2008-04-08	11
+439	val_439	2008-04-08	11
+443	val_443	2008-04-08	11
+444	val_444	2008-04-08	11
+446	val_446	2008-04-08	11
+448	val_448	2008-04-08	11
+449	val_449	2008-04-08	11
+452	val_452	2008-04-08	11
+453	val_453	2008-04-08	11
+454	val_454	2008-04-08	11
+454	val_454	2008-04-08	11
+454	val_454	2008-04-08	11
+455	val_455	2008-04-08	11
+457	val_457	2008-04-08	11
+458	val_458	2008-04-08	11
+458	val_458	2008-04-08	11
+459	val_459	2008-04-08	11
+459	val_459	2008-04-08	11
+460	val_460	2008-04-08	11
+462	val_462	2008-04-08	11
+462	val_462	2008-04-08	11
+463	val_463	2008-04-08	11
+463	val_463	2008-04-08	11
+466	val_466	2008-04-08	11
+466	val_466	2008-04-08	11
+466	val_466	2008-04-08	11
+467	val_467	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+470	val_470	2008-04-08	11
+472	val_472	2008-04-08	11
+475	val_475	2008-04-08	11
+477	val_477	2008-04-08	11
+478	val_478	2008-04-08	11
+478	val_478	2008-04-08	11
+479	val_479	2008-04-08	11
+480	val_480	2008-04-08	11
+480	val_480	2008-04-08	11
+480	val_480	2008-04-08	11
+481	val_481	2008-04-08	11
+482	val_482	2008-04-08	11
+483	val_483	2008-04-08	11
+484	val_484	2008-04-08	11
+485	val_485	2008-04-08	11
+487	val_487	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+490	val_490	2008-04-08	11
+491	val_491	2008-04-08	11
+492	val_492	2008-04-08	11
+492	val_492	2008-04-08	11
+493	val_493	2008-04-08	11
+494	val_494	2008-04-08	11
+495	val_495	2008-04-08	11
+496	val_496	2008-04-08	11
+497	val_497	2008-04-08	11
+498	val_498	2008-04-08	11
+498	val_498	2008-04-08	11
+498	val_498	2008-04-08	11
+0	val_0	2008-04-08	12
+0	val_0	2008-04-08	12
+0	val_0	2008-04-08	12
+2	val_2	2008-04-08	12
+4	val_4	2008-04-08	12
+5	val_5	2008-04-08	12
+5	val_5	2008-04-08	12
+5	val_5	2008-04-08	12
+8	val_8	2008-04-08	12
+9	val_9	2008-04-08	12
+10	val_10	2008-04-08	12
+11	val_11	2008-04-08	12
+12	val_12	2008-04-08	12
+12	val_12	2008-04-08	12
+15	val_15	2008-04-08	12
+15	val_15	2008-04-08	12
+17	val_17	2008-04-08	12
+18	val_18	2008-04-08	12
+18	val_18	2008-04-08	12
+19	val_19	2008-04-08	12
+20	val_20	2008-04-08	12
+24	val_24	2008-04-08	12
+24	val_24	2008-04-08	12
+26	val_26	2008-04-08	12
+26	val_26	2008-04-08	12
+27	val_27	2008-04-08	12
+28	val_28	2008-04-08	12
+30	val_30	2008-04-08	12
+33	val_33	2008-04-08	12
+34	val_34	2008-04-08	12
+35	val_35	2008-04-08	12
+35	val_35	2008-04-08	12
+35	val_35	2008-04-08	12
+37	val_37	2008-04-08	12
+37	val_37	2008-04-08	12
+41	val_41	2008-04-08	12
+42	val_42	2008-04-08	12
+42	val_42	2008-04-08	12
+43	val_43	2008-04-08	12
+44	val_44	2008-04-08	12
+47	val_47	2008-04-08	12
+51	val_51	2008-04-08	12
+51	val_51	2008-04-08	12
+53	val_53	2008-04-08	12
+54	val_54	2008-04-08	12
+57	val_57	2008-04-08	12
+58	val_58	2008-04-08	12
+58	val_58	2008-04-08	12
+64	val_64	2008-04-08	12
+65	val_65	2008-04-08	12
+66	val_66	2008-04-08	12
+67	val_67	2008-04-08	12
+67	val_67	2008-04-08	12
+69	val_69	2008-04-08	12
+70	val_70	2008-04-08	12
+70	val_70	2008-04-08	12
+70	val_70	2008-04-08	12
+72	val_72	2008-04-08	12
+72	val_72	2008-04-08	12
+74	val_74	2008-04-08	12
+76	val_76	2008-04-08	12
+76	val_76	2008-04-08	12
+77	val_77	2008-04-08	12
+78	val_78	2008-04-08	12
+80	val_80	2008-04-08	12
+82	val_82	2008-04-08	12
+83	val_83	2008-04-08	12
+83	val_83	2008-04-08	12
+84	val_84	2008-04-08	12
+84	val_84	2008-04-08	12
+85	val_85	2008-04-08	12
+86	val_86	2008-04-08	12
+87	val_87	2008-04-08	12
+90	val_90	2008-04-08	12
+90	val_90	2008-04-08	12
+90	val_90	2008-04-08	12
+92	val_92	2008-04-08	12
+95	val_95	2008-04-08	12
+95	val_95	2008-04-08	12
+96	val_96	2008-04-08	12
+97	val_97	2008-04-08	12
+97	val_97	2008-04-08	12
+98	val_98	2008-04-08	12
+98	val_98	2008-04-08	12
+100	val_100	2008-04-08	12
+100	val_100	2008-04-08	12
+103	val_103	2008-04-08	12
+103	val_103	2008-04-08	12
+104	val_104	2008-04-08	12
+104	val_104	2008-04-08	12
+105	val_105	2008-04-08	12
+111	val_111	2008-04-08	12
+113	val_113	2008-04-08	12
+113	val_113	2008-04-08	12
+114	val_114	2008-04-08	12
+116	val_116	2008-04-08	12
+118	val_118	2008-04-08	12
+118	val_118	2008-04-08	12
+119	val_119	2008-04-08	12
+119	val_119	2008-04-08	12
+119	val_119	2008-04-08	12
+120	val_120	2008-04-08	12
+120	val_120	2008-04-08	12
+125	val_125	2008-04-08	12
+125	val_125	2008-04-08	12
+126	val_126	2008-04-08	12
+128	val_128	2008-04-08	12
+128	val_128	2008-04-08	12
+128	val_128	2008-04-08	12
+129	val_129	2008-04-08	12
+129	val_129	2008-04-08	12
+131	val_131	2008-04-08	12
+133	val_133	2008-04-08	12
+134	val_134	2008-04-08	12
+134	val_134	2008-04-08	12
+136	val_136	2008-04-08	12
+137	val_137	2008-04-08	12
+137	val_137	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+143	val_143	2008-04-08	12
+145	val_145	2008-04-08	12
+146	val_146	2008-04-08	12
+146	val_146	2008-04-08	12
+149	val_149	2008-04-08	12
+149	val_149	2008-04-08	12
+150	val_150	2008-04-08	12
+152	val_152	2008-04-08	12
+152	val_152	2008-04-08	12
+153	val_153	2008-04-08	12
+155	val_155	2008-04-08	12
+156	val_156	2008-04-08	12
+157	val_157	2008-04-08	12
+158	val_158	2008-04-08	12
+160	val_160	2008-04-08	12
+162	val_162	2008-04-08	12
+163	val_163	2008-04-08	12
+164	val_164	2008-04-08	12
+164	val_164	2008-04-08	12
+165	val_165	2008-04-08	12
+165	val_165	2008-04-08	12
+166	val_166	2008-04-08	12
+167	val_167	2008-04-08	12
+167	val_167	2008-04-08	12
+167	val_167	2008-04-08	12
+168	val_168	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+170	val_170	2008-04-08	12
+172	val_172	2008-04-08	12
+172	val_172	2008-04-08	12
+174	val_174	2008-04-08	12
+174	val_174	2008-04-08	12
+175	val_175	2008-04-08	12
+175	val_175	2008-04-08	12
+176	val_176	2008-04-08	12
+176	val_176	2008-04-08	12
+177	val_177	2008-04-08	12
+178	val_178	2008-04-08	12
+179	val_179	2008-04-08	12
+179	val_179	2008-04-08	12
+180	val_180	2008-04-08	12
+181	val_181	2008-04-08	12
+183	val_183	2008-04-08	12
+186	val_186	2008-04-08	12
+187	val_187	2008-04-08	12
+187	val_187	2008-04-08	12
+187	val_187	2008-04-08	12
+189	val_189	2008-04-08	12
+190	val_190	2008-04-08	12
+191	val_191	2008-04-08	12
+191	val_191	2008-04-08	12
+192	val_192	2008-04-08	12
+193	val_193	2008-04-08	12
+193	val_193	2008-04-08	12
+193	val_193	2008-04-08	12
+194	val_194	2008-04-08	12
+195	val_195	2008-04-08	12
+195	val_195	2008-04-08	12
+196	val_196	2008-04-08	12
+197	val_197	2008-04-08	12
+197	val_197	2008-04-08	12
+199	val_199	2008-04-08	12
+199	val_199	2008-04-08	12
+199	val_199	2008-04-08	12
+200	val_200	2008-04-08	12
+200	val_200	2008-04-08	12
+201	val_201	2008-04-08	12
+202	val_202	2008-04-08	12
+203	val_203	2008-04-08	12
+203	val_203	2008-04-08	12
+205	val_205	2008-04-08	12
+205	val_205	2008-04-08	12
+207	val_207	2008-04-08	12
+207	val_207	2008-04-08	12
+208	val_208	2008-04-08	12
+208	val_208	2008-04-08	12
+208	val_208	2008-04-08	12
+209	val_209	2008-04-08	12
+209	val_209	2008-04-08	12
+213	val_213	2008-04-08	12
+213	val_213	2008-04-08	12
+214	val_214	2008-04-08	12
+216	val_216	2008-04-08	12
+216	val_216	2008-04-08	12
+217	val_217	2008-04-08	12
+217	val_217	2008-04-08	12
+218	val_218	2008-04-08	12
+219	val_219	2008-04-08	12
+219	val_219	2008-04-08	12
+221	val_221	2008-04-08	12
+221	val_221	2008-04-08	12
+222	val_222	2008-04-08	12
+223	val_223	2008-04-08	12
+223	val_223	2008-04-08	12
+224	val_224	2008-04-08	12
+224	val_224	2008-04-08	12
+226	val_226	2008-04-08	12
+228	val_228	2008-04-08	12
+229	val_229	2008-04-08	12
+229	val_229	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+233	val_233	2008-04-08	12
+233	val_233	2008-04-08	12
+235	val_235	2008-04-08	12
+237	val_237	2008-04-08	12
+237	val_237	2008-04-08	12
+238	val_238	2008-04-08	12
+238	val_238	2008-04-08	12
+239	val_239	2008-04-08	12
+239	val_239	2008-04-08	12
+241	val_241	2008-04-08	12
+242	val_242	2008-04-08	12
+242	val_242	2008-04-08	12
+244	val_244	2008-04-08	12
+247	val_247	2008-04-08	12
+248	val_248	2008-04-08	12
+249	val_249	2008-04-08	12
+252	val_252	2008-04-08	12
+255	val_255	2008-04-08	12
+255	val_255	2008-04-08	12
+256	val_256	2008-04-08	12
+256	val_256	2008-04-08	12
+257	val_257	2008-04-08	12
+258	val_258	2008-04-08	12
+260	val_260	2008-04-08	12
+262	val_262	2008-04-08	12
+263	val_263	2008-04-08	12
+265	val_265	2008-04-08	12
+265	val_265	2008-04-08	12
+266	val_266	2008-04-08	12
+272	val_272	2008-04-08	12
+272	val_272	2008-04-08	12
+273	val_273	2008-04-08	12
+273	val_273	2008-04-08	12
+273	val_273	2008-04-08	12
+274	val_274	2008-04-08	12
+275	val_275	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+278	val_278	2008-04-08	12
+278	val_278	2008-04-08	12
+280	val_280	2008-04-08	12
+280	val_280	2008-04-08	12
+281	val_281	2008-04-08	12
+281	val_281	2008-04-08	12
+282	val_282	2008-04-08	12
+282	val_282	2008-04-08	12
+283	val_283	2008-04-08	12
+284	val_284	2008-04-08	12
+285	val_285	2008-04-08	12
+286	val_286	2008-04-08	12
+287	val_287	2008-04-08	12
+288	val_288	2008-04-08	12
+288	val_288	2008-04-08	12
+289	val_289	2008-04-08	12
+291	val_291	2008-04-08	12
+292	val_292	2008-04-08	12
+296	val_296	2008-04-08	12
+298	val_298	2008-04-08	12
+298	val_298	2008-04-08	12
+298	val_298	2008-04-08	12
+302	val_302	2008-04-08	12
+305	val_305	2008-04-08	12
+306	val_306	2008-04-08	12
+307	val_307	2008-04-08	12
+307	val_307	2008-04-08	12
+308	val_308	2008-04-08	12
+309	val_309	2008-04-08	12
+309	val_309	2008-04-08	12
+310	val_310	2008-04-08	12
+311	val_311	2008-04-08	12
+311	val_311	2008-04-08	12
+311	val_311	2008-04-08	12
+315	val_315	2008-04-08	12
+316	val_316	2008-04-08	12
+316	val_316	2008-04-08	12
+316	val_316	2008-04-08	12
+317	val_317	2008-04-08	12
+317	val_317	2008-04-08	12
+318	val_318	2008-04-08	12
+318	val_318	2008-04-08	12
+318	val_318	2008-04-08	12
+321	val_321	2008-04-08	12
+321	val_321	2008-04-08	12
+322	val_322	2008-04-08	12
+322	val_322	2008-04-08	12
+323	val_323	2008-04-08	12
+325	val_325	2008-04-08	12
+325	val_325	2008-04-08	12
+327	val_327	2008-04-08	12
+327	val_327	2008-04-08	12
+327	val_327	2008-04-08	12
+331	val_331	2008-04-08	12
+331	val_331	2008-04-08	12
+332	val_332	2008-04-08	12
+333	val_333	2008-04-08	12
+333	val_333	2008-04-08	12
+335	val_335	2008-04-08	12
+336	val_336	2008-04-08	12
+338	val_338	2008-04-08	12
+339	val_339	2008-04-08	12
+341	val_341	2008-04-08	12
+342	val_342	2008-04-08	12
+342	val_342	2008-04-08	12
+344	val_344	2008-04-08	12
+344	val_344	2008-04-08	12
+345	val_345	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+351	val_351	2008-04-08	12
+353	val_353	2008-04-08	12
+353	val_353	2008-04-08	12
+356	val_356	2008-04-08	12
+360	val_360	2008-04-08	12
+362	val_362	2008-04-08	12
+364	val_364	2008-04-08	12
+365	val_365	2008-04-08	12
+366	val_366	2008-04-08	12
+367	val_367	2008-04-08	12
+367	val_367	2008-04-08	12
+368	val_368	2008-04-08	12
+369	val_369	2008-04-08	12
+369	val_369	2008-04-08	12
+369	val_369	2008-04-08	12
+373	val_373	2008-04-08	12
+374	val_374	2008-04-08	12
+375	val_375	2008-04-08	12
+377	val_377	2008-04-08	12
+378	val_378	2008-04-08	12
+379	val_379	2008-04-08	12
+382	val_382	2008-04-08	12
+382	val_382	2008-04-08	12
+384	val_384	2008-04-08	12
+384	val_384	2008-04-08	12
+384	val_384	2008-04-08	12
+386	val_386	2008-04-08	12
+389	val_389	2008-04-08	12
+392	val_392	2008-04-08	12
+393	val_393	2008-04-08	12
+394	val_394	2008-04-08	12
+395	val_395	2008-04-08	12
+395	val_395	2008-04-08	12
+396	val_396	2008-04-08	12
+396	val_396	2008-04-08	12
+396	val_396	2008-04-08	12
+397	val_397	2008-04-08	12
+397	val_397	2008-04-08	12
+399	val_399	2008-04-08	12
+399	val_399	2008-04-08	12
+400	val_400	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+402	val_402	2008-04-08	12
+403	val_403	2008-04-08	12
+403	val_403	2008-04-08	12
+403	val_403	2008-04-08	12
+404	val_404	2008-04-08	12
+404	val_404	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+407	val_407	2008-04-08	12
+409	val_409	2008-04-08	12
+409	val_409	2008-04-08	12
+409	val_409	2008-04-08	12
+411	val_411	2008-04-08	12
+413	val_413	2008-04-08	12
+413	val_413	2008-04-08	12
+414	val_414	2008-04-08	12
+414	val_414	2008-04-08	12
+417	val_417	2008-04-08	12
+417	val_417	2008-04-08	12
+417	val_417	2008-04-08	12
+418	val_418	2008-04-08	12
+419	val_419	2008-04-08	12
+421	val_421	2008-04-08	12
+424	val_424	2008-04-08	12
+424	val_424	2008-04-08	12
+427	val_427	2008-04-08	12
+429	val_429	2008-04-08	12
+429	val_429	2008-04-08	12
+430	val_430	2008-04-08	12
+430	val_430	2008-04-08	12
+430	val_430	2008-04-08	12
+431	val_431	2008-04-08	12
+431	val_431	2008-04-08	12
+431	val_431	2008-04-08	12
+432	val_432	2008-04-08	12
+435	val_435	2008-04-08	12
+436	val_436	2008-04-08	12
+437	val_437	2008-04-08	12
+438	val_438	2008-04-08	12
+438	val_438	2008-04-08	12
+438	val_438	2008-04-08	12
+439	val_439	2008-04-08	12
+439	val_439	2008-04-08	12
+443	val_443	2008-04-08	12
+444	val_444	2008-04-08	12
+446	val_446	2008-04-08	12
+448	val_448	2008-04-08	12
+449	val_449	2008-04-08	12
+452	val_452	2008-04-08	12
+453	val_453	2008-04-08	12
+454	val_454	2008-04-08	12
+454	val_454	2008-04-08	12
+454	val_454	2008-04-08	12
+455	val_455	2008-04-08	12
+457	val_457	2008-04-08	12
+458	val_458	2008-04-08	12
+458	val_458	2008-04-08	12
+459	val_459	2008-04-08	12
+459	val_459	2008-04-08	12
+460	val_460	2008-04-08	12
+462	val_462	2008-04-08	12
+462	val_462	2008-04-08	12
+463	val_463	2008-04-08	12
+463	val_463	2008-04-08	12
+466	val_466	2008-04-08	12
+466	val_466	2008-04-08	12
+466	val_466	2008-04-08	12
+467	val_467	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+470	val_470	2008-04-08	12
+472	val_472	2008-04-08	12
+475	val_475	2008-04-08	12
+477	val_477	2008-04-08	12
+478	val_478	2008-04-08	12
+478	val_478	2008-04-08	12
+479	val_479	2008-04-08	12
+480	val_480	2008-04-08	12
+480	val_480	2008-04-08	12
+480	val_480	2008-04-08	12
+481	val_481	2008-04-08	12
+482	val_482	2008-04-08	12
+483	val_483	2008-04-08	12
+484	val_484	2008-04-08	12
+485	val_485	2008-04-08	12
+487	val_487	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+490	val_490	2008-04-08	12
+491	val_491	2008-04-08	12
+492	val_492	2008-04-08	12
+492	val_492	2008-04-08	12
+493	val_493	2008-04-08	12
+494	val_494	2008-04-08	12
+495	val_495	2008-04-08	12
+496	val_496	2008-04-08	12
+497	val_497	2008-04-08	12
+498	val_498	2008-04-08	12
+498	val_498	2008-04-08	12
+498	val_498	2008-04-08	12
+0	val_0	2008-04-08	existing_value
+0	val_0	2008-04-08	existing_value
+0	val_0	2008-04-08	existing_value
+2	val_2	2008-04-08	existing_value
+4	val_4	2008-04-08	existing_value
+5	val_5	2008-04-08	existing_value
+5	val_5	2008-04-08	existing_value
+5	val_5	2008-04-08	existing_value
+8	val_8	2008-04-08	existing_value
+9	val_9	2008-04-08	existing_value
+10	val_10	2008-04-08	existing_value
+11	val_11	2008-04-08	existing_value
+12	val_12	2008-04-08	existing_value
+12	val_12	2008-04-08	existing_value
+15	val_15	2008-04-08	existing_value
+15	val_15	2008-04-08	existing_value
+17	val_17	2008-04-08	existing_value
+18	val_18	2008-04-08	existing_value
+18	val_18	2008-04-08	existing_value
+19	val_19	2008-04-08	existing_value
+20	val_20	2008-04-08	existing_value
+24	val_24	2008-04-08	existing_value
+24	val_24	2008-04-08	existing_value
+26	val_26	2008-04-08	existing_value
+26	val_26	2008-04-08	existing_value
+27	val_27	2008-04-08	existing_value
+28	val_28	2008-04-08	existing_value
+30	val_30	2008-04-08	existing_value
+33	val_33	2008-04-08	existing_value
+34	val_34	2008-04-08	existing_value
+35	val_35	2008-04-08	existing_value
+35	val_35	2008-04-08	existing_value
+35	val_35	2008-04-08	existing_value
+37	val_37	2008-04-08	existing_value
+37	val_37	2008-04-08	existing_value
+41	val_41	2008-04-08	existing_value
+42	val_42	2008-04-08	existing_value
+42	val_42	2008-04-08	existing_value
+43	val_43	2008-04-08	existing_value
+44	val_44	2008-04-08	existing_value
+47	val_47	2008-04-08	existing_value
+51	val_51	2008-04-08	existing_value
+51	val_51	2008-04-08	existing_value
+53	val_53	2008-04-08	existing_value
+54	val_54	2008-04-08	existing_value
+57	val_57	2008-04-08	existing_value
+58	val_58	2008-04-08	existing_value
+58	val_58	2008-04-08	existing_value
+64	val_64	2008-04-08	existing_value
+65	val_65	2008-04-08	existing_value
+66	val_66	2008-04-08	existing_value
+67	val_67	2008-04-08	existing_value
+67	val_67	2008-04-08	existing_value
+69	val_69	2008-04-08	existing_value
+70	val_70	2008-04-08	existing_value
+70	val_70	2008-04-08	existing_value
+70	val_70	2008-04-08	existing_value
+72	val_72	2008-04-08	existing_value
+72	val_72	2008-04-08	existing_value
+74	val_74	2008-04-08	existing_value
+76	val_76	2008-04-08	existing_value
+76	val_76	2008-04-08	existing_value
+77	val_77	2008-04-08	existing_value
+78	val_78	2008-04-08	existing_value
+80	val_80	2008-04-08	existing_value
+82	val_82	2008-04-08	existing_value
+83	val_83	2008-04-08	existing_value
+83	val_83	2008-04-08	existing_value
+84	val_84	2008-04-08	existing_value
+84	val_84	2008-04-08	existing_value
+85	val_85	2008-04-08	existing_value
+86	val_86	2008-04-08	existing_value
+87	val_87	2008-04-08	existing_value
+90	val_90	2008-04-08	existing_value
+90	val_90	2008-04-08	existing_value
+90	val_90	2008-04-08	existing_value
+92	val_92	2008-04-08	existing_value
+95	val_95	2008-04-08	existing_value
+95	val_95	2008-04-08	existing_value
+96	val_96	2008-04-08	existing_value
+97	val_97	2008-04-08	existing_value
+97	val_97	2008-04-08	existing_value
+98	val_98	2008-04-08	existing_value
+98	val_98	2008-04-08	existing_value
+100	val_100	2008-04-08	existing_value
+100	val_100	2008-04-08	existing_value
+103	val_103	2008-04-08	existing_value
+103	val_103	2008-04-08	existing_value
+104	val_104	2008-04-08	existing_value
+104	val_104	2008-04-08	existing_value
+105	val_105	2008-04-08	existing_value
+111	val_111	2008-04-08	existing_value
+113	val_113	2008-04-08	existing_value
+113	val_113	2008-04-08	existing_value
+114	val_114	2008-04-08	existing_value
+116	val_116	2008-04-08	existing_value
+118	val_118	2008-04-08	existing_value
+118	val_118	2008-04-08	existing_value
+119	val_119	2008-04-08	existing_value
+119	val_119	2008-04-08	existing_value
+119	val_119	2008-04-08	existing_value
+120	val_120	2008-04-08	existing_value
+120	val_120	2008-04-08	existing_value
+125	val_125	2008-04-08	existing_value
+125	val_125	2008-04-08	existing_value
+126	val_126	2008-04-08	existing_value
+128	val_128	2008-04-08	existing_value
+128	val_128	2008-04-08	existing_value
+128	val_128	2008-04-08	existing_value
+129	val_129	2008-04-08	existing_value
+129	val_129	2008-04-08	existing_value
+131	val_131	2008-04-08	existing_value
+133	val_133	2008-04-08	existing_value
+134	val_134	2008-04-08	existing_value
+134	val_134	2008-04-08	existing_value
+136	val_136	2008-04-08	existing_value
+137	val_137	2008-04-08	existing_value
+137	val_137	2008-04-08	existing_value
+138	val_138	2008-04-08	existing_value
+138	val_138	2008-04-08	existing_value
+138	val_138	2008-04-08	existing_value
+138	val_138	2008-04-08	existing_value
+143	val_143	2008-04-08	existing_value
+145	val_145	2008-04-08	existing_value
+146	val_146	2008-04-08	existing_value
+146	val_146	2008-04-08	existing_value
+149	val_149	2008-04-08	existing_value
+149	val_149	2008-04-08	existing_value
+150	val_150	2008-04-08	existing_value
+152	val_152	2008-04-08	existing_value
+152	val_152	2008-04-08	existing_value
+153	val_153	2008-04-08	existing_value
+155	val_155	2008-04-08	existing_value
+156	val_156	2008-04-08	existing_value
+157	val_157	2008-04-08	existing_value
+158	val_158	2008-04-08	existing_value
+160	val_160	2008-04-08	existing_value
+162	val_162	2008-04-08	existing_value
+163	val_163	2008-04-08	existing_value
+164	val_164	2008-04-08	existing_value
+164	val_164	2008-04-08	existing_value
+165	val_165	2008-04-08	existing_value
+165	val_165	2008-04-08	existing_value
+166	val_166	2008-04-08	existing_value
+167	val_167	2008-04-08	existing_value
+167	val_167	2008-04-08	existing_value
+167	val_167	2008-04-08	existing_value
+168	val_168	2008-04-08	existing_value
+169	val_169	2008-04-08	existing_value
+169	val_169	2008-04-08	existing_value
+169	val_169	2008-04-08	existing_value
+169	val_169	2008-04-08	existing_value
+170	val_170	2008-04-08	existing_value
+172	val_172	2008-04-08	existing_value
+172	val_172	2008-04-08	existing_value
+174	val_174	2008-04-08	existing_value
+174	val_174	2008-04-08	existing_value
+175	val_175	2008-04-08	existing_value
+175	val_175	2008-04-08	existing_value
+176	val_176	2008-04-08	existing_value
+176	val_176	2008-04-08	existing_value
+177	val_177	2008-04-08	existing_value
+178	val_178	2008-04-08	existing_value
+179	val_179	2008-04-08	existing_value
+179	val_179	2008-04-08	existing_value
+180	val_180	2008-04-08	existing_value
+181	val_181	2008-04-08	existing_value
+183	val_183	2008-04-08	existing_value
+186	val_186	2008-04-08	existing_value
+187	val_187	2008-04-08	existing_value
+187	val_187	2008-04-08	existing_value
+187	val_187	2008-04-08	existing_value
+189	val_189	2008-04-08	existing_value
+190	val_190	2008-04-08	existing_value
+191	val_191	2008-04-08	existing_value
+191	val_191	2008-04-08	existing_value
+192	val_192	2008-04-08	existing_value
+193	val_193	2008-04-08	existing_value
+193	val_193	2008-04-08	existing_value
+193	val_193	2008-04-08	existing_value
+194	val_194	2008-04-08	existing_value
+195	val_195	2008-04-08	existing_value
+195	val_195	2008-04-08	existing_value
+196	val_196	2008-04-08	existing_value
+197	val_197	2008-04-08	existing_value
+197	val_197	2008-04-08	existing_value
+199	val_199	2008-04-08	existing_value
+199	val_199	2008-04-08	existing_value
+199	val_199	2008-04-08	existing_value
+200	val_200	2008-04-08	existing_value
+200	val_200	2008-04-08	existing_value
+201	val_201	2008-04-08	existing_value
+202	val_202	2008-04-08	existing_value
+203	val_203	2008-04-08	existing_value
+203	val_203	2008-04-08	existing_value
+205	val_205	2008-04-08	existing_value
+205	val_205	2008-04-08	existing_value
+207	val_207	2008-04-08	existing_value
+207	val_207	2008-04-08	existing_value
+208	val_208	2008-04-08	existing_value
+208	val_208	2008-04-08	existing_value
+208	val_208	2008-04-08	existing_value
+209	val_209	2008-04-08	existing_value
+209	val_209	2008-04-08	existing_value
+213	val_213	2008-04-08	existing_value
+213	val_213	2008-04-08	existing_value
+214	val_214	2008-04-08	existing_value
+216	val_216	2008-04-08	existing_value
+216	val_216	2008-04-08	existing_value
+217	val_217	2008-04-08	existing_value
+217	val_217	2008-04-08	existing_value
+218	val_218	2008-04-08	existing_value
+219	val_219	2008-04-08	existing_value
+219	val_219	2008-04-08	existing_value
+221	val_221	2008-04-08	existing_value
+221	val_221	2008-04-08	existing_value
+222	val_222	2008-04-08	existing_value
+223	val_223	2008-04-08	existing_value
+223	val_223	2008-04-08	existing_value
+224	val_224	2008-04-08	existing_value
+224	val_224	2008-04-08	existing_value
+226	val_226	2008-04-08	existing_value
+228	val_228	2008-04-08	existing_value
+229	val_229	2008-04-08	existing_value
+229	val_229	2008-04-08	existing_value
+230	val_230	2008-04-08	existing_value
+230	val_230	2008-04-08	existing_value
+230	val_230	2008-04-08	existing_value
+230	val_230	2008-04-08	existing_value
+230	val_230	2008-04-08	existing_value
+233	val_233	2008-04-08	existing_value
+233	val_233	2008-04-08	existing_value
+235	val_235	2008-04-08	existing_value
+237	val_237	2008-04-08	existing_value
+237	val_237	2008-04-08	existing_value
+238	val_238	2008-04-08	existing_value
+238	val_238	2008-04-08	existing_value
+239	val_239	2008-04-08	existing_value
+239	val_239	2008-04-08	existing_value
+241	val_241	2008-04-08	existing_value
+242	val_242	2008-04-08	existing_value
+242	val_242	2008-04-08	existing_value
+244	val_244	2008-04-08	existing_value
+247	val_247	2008-04-08	existing_value
+248	val_248	2008-04-08	existing_value
+249	val_249	2008-04-08	existing_value
+252	val_252	2008-04-08	existing_value
+255	val_255	2008-04-08	existing_value
+255	val_255	2008-04-08	existing_value
+256	val_256	2008-04-08	existing_value
+256	val_256	2008-04-08	existing_value
+257	val_257	2008-04-08	existing_value
+258	val_258	2008-04-08	existing_value
+260	val_260	2008-04-08	existing_value
+262	val_262	2008-04-08	existing_value
+263	val_263	2008-04-08	existing_value
+265	val_265	2008-04-08	existing_value
+265	val_265	2008-04-08	existing_value
+266	val_266	2008-04-08	existing_value
+272	val_272	2008-04-08	existing_value
+272	val_272	2008-04-08	existing_value
+273	val_273	2008-04-08	existing_value
+273	val_273	2008-04-08	existing_value
+273	val_273	2008-04-08	existing_value
+274	val_274	2008-04-08	existing_value
+275	val_275	2008-04-08	existing_value
+277	val_277	2008-04-08	existing_value
+277	val_277	2008-04-08	existing_value
+277	val_277	2008-04-08	existing_value
+277	val_277	2008-04-08	existing_value
+278	val_278	2008-04-08	existing_value
+278	val_278	2008-04-08	existing_value
+280	val_280	2008-04-08	existing_value
+280	val_280	2008-04-08	existing_value
+281	val_281	2008-04-08	existing_value
+281	val_281	2008-04-08	existing_value
+282	val_282	2008-04-08	existing_value
+282	val_282	2008-04-08	existing_value
+283	val_283	2008-04-08	existing_value
+284	val_284	2008-04-08	existing_value
+285	val_285	2008-04-08	existing_value
+286	val_286	2008-04-08	existing_value
+287	val_287	2008-04-08	existing_value
+288	val_288	2008-04-08	existing_value
+288	val_288	2008-04-08	existing_value
+289	val_289	2008-04-08	existing_value
+291	val_291	2008-04-08	existing_value
+292	val_292	2008-04-08	existing_value
+296	val_296	2008-04-08	existing_value
+298	val_298	2008-04-08	existing_value
+298	val_298	2008-04-08	existing_value
+298	val_298	2008-04-08	existing_value
+302	val_302	2008-04-08	existing_value
+305	val_305	2008-04-08	existing_value
+306	val_306	2008-04-08	existing_value
+307	val_307	2008-04-08	existing_value
+307	val_307	2008-04-08	existing_value
+308	val_308	2008-04-08	existing_value
+309	val_309	2008-04-08	existing_value
+309	val_309	2008-04-08	existing_value
+310	val_310	2008-04-08	existing_value
+311	val_311	2008-04-08	existing_value
+311	val_311	2008-04-08	existing_value
+311	val_311	2008-04-08	existing_value
+315	val_315	2008-04-08	existing_value
+316	val_316	2008-04-08	existing_value
+316	val_316	2008-04-08	existing_value
+316	val_316	2008-04-08	existing_value
+317	val_317	2008-04-08	existing_value
+317	val_317	2008-04-08	existing_value
+318	val_318	2008-04-08	existing_value
+318	val_318	2008-04-08	existing_value
+318	val_318	2008-04-08	existing_value
+321	val_321	2008-04-08	existing_value
+321	val_321	2008-04-08	existing_value
+322	val_322	2008-04-08	existing_value
+322	val_322	2008-04-08	existing_value
+323	val_323	2008-04-08	existing_value
+325	val_325	2008-04-08	existing_value
+325	val_325	2008-04-08	existing_value
+327	val_327	2008-04-08	existing_value
+327	val_327	2008-04-08	existing_value
+327	val_327	2008-04-08	existing_value
+331	val_331	2008-04-08	existing_value
+331	val_331	2008-04-08	existing_value
+332	val_332	2008-04-08	existing_value
+333	val_333	2008-04-08	existing_value
+333	val_333	2008-04-08	existing_value
+335	val_335	2008-04-08	existing_value
+336	val_336	2008-04-08	existing_value
+338	val_338	2008-04-08	existing_value
+339	val_339	2008-04-08	existing_value
+341	val_341	2008-04-08	existing_value
+342	val_342	2008-04-08	existing_value
+342	val_342	2008-04-08	existing_value
+344	val_344	2008-04-08	existing_value
+344	val_344	2008-04-08	existing_value
+345	val_345	2008-04-08	existing_value
+348	val_348	2008-04-08	existing_value
+348	val_348	2008-04-08	existing_value
+348	val_348	2008-04-08	existing_value
+348	val_348	2008-04-08	existing_value
+348	val_348	2008-04-08	existing_value
+351	val_351	2008-04-08	existing_value
+353	val_353	2008-04-08	existing_value
+353	val_353	2008-04-08	existing_value
+356	val_356	2008-04-08	existing_value
+360	val_360	2008-04-08	existing_value
+362	val_362	2008-04-08	existing_value
+364	val_364	2008-04-08	existing_value
+365	val_365	2008-04-08	existing_value
+366	val_366	2008-04-08	existing_value
+367	val_367	2008-04-08	existing_value
+367	val_367	2008-04-08	existing_value
+368	val_368	2008-04-08	existing_value
+369	val_369	2008-04-08	existing_value
+369	val_369	2008-04-08	existing_value
+369	val_369	2008-04-08	existing_value
+373	val_373	2008-04-08	existing_value
+374	val_374	2008-04-08	existing_value
+375	val_375	2008-04-08	existing_value
+377	val_377	2008-04-08	existing_value
+378	val_378	2008-04-08	existing_value
+379	val_379	2008-04-08	existing_value
+382	val_382	2008-04-08	existing_value
+382	val_382	2008-04-08	existing_value
+384	val_384	2008-04-08	existing_value
+384	val_384	2008-04-08	existing_value
+384	val_384	2008-04-08	existing_value
+386	val_386	2008-04-08	existing_value
+389	val_389	2008-04-08	existing_value
+392	val_392	2008-04-08	existing_value
+393	val_393	2008-04-08	existing_value
+394	val_394	2008-04-08	existing_value
+395	val_395	2008-04-08	existing_value
+395	val_395	2008-04-08	existing_value
+396	val_396	2008-04-08	existing_value
+396	val_396	2008-04-08	existing_value
+396	val_396	2008-04-08	existing_value
+397	val_397	2008-04-08	existing_value
+397	val_397	2008-04-08	existing_value
+399	val_399	2008-04-08	existing_value
+399	val_399	2008-04-08	existing_value
+400	val_400	2008-04-08	existing_value
+401	val_401	2008-04-08	existing_value
+401	val_401	2008-04-08	existing_value
+401	val_401	2008-04-08	existing_value
+401	val_401	2008-04-08	existing_value
+401	val_401	2008-04-08	existing_value
+402	val_402	2008-04-08	existing_value
+403	val_403	2008-04-08	existing_value
+403	val_403	2008-04-08	existing_value
+403	val_403	2008-04-08	existing_value
+404	val_404	2008-04-08	existing_value
+404	val_404	2008-04-08	existing_value
+406	val_406	2008-04-08	existing_value
+406	val_406	2008-04-08	existing_value
+406	val_406	2008-04-08	existing_value
+406	val_406	2008-04-08	existing_value
+407	val_407	2008-04-08	existing_value
+409	val_409	2008-04-08	existing_value
+409	val_409	2008-04-08	existing_value
+409	val_409	2008-04-08	existing_value
+411	val_411	2008-04-08	existing_value
+413	val_413	2008-04-08	existing_value
+413	val_413	2008-04-08	existing_value
+414	val_414	2008-04-08	existing_value
+414	val_414	2008-04-08	existing_value
+417	val_417	2008-04-08	existing_value
+417	val_417	2008-04-08	existing_value
+417	val_417	2008-04-08	existing_value
+418	val_418	2008-04-08	existing_value
+419	val_419	2008-04-08	existing_value
+421	val_421	2008-04-08	existing_value
+424	val_424	2008-04-08	existing_value
+424	val_424	2008-04-08	existing_value
+427	val_427	2008-04-08	existing_value
+429	val_429	2008-04-08	existing_value
+429	val_429	2008-04-08	existing_value
+430	val_430	2008-04-08	existing_value
+430	val_430	2008-04-08	existing_value
+430	val_430	2008-04-08	existing_value
+431	val_431	2008-04-08	existing_value
+431	val_431	2008-04-08	existing_value
+431	val_431	2008-04-08	existing_value
+432	val_432	2008-04-08	existing_value
+435	val_435	2008-04-08	existing_value
+436	val_436	2008-04-08	existing_value
+437	val_437	2008-04-08	existing_value
+438	val_438	2008-04-08	existing_value
+438	val_438	2008-04-08	existing_value
+438	val_438	2008-04-08	existing_value
+439	val_439	2008-04-08	existing_value
+439	val_439	2008-04-08	existing_value
+443	val_443	2008-04-08	existing_value
+444	val_444	2008-04-08	existing_value
+446	val_446	2008-04-08	existing_value
+448	val_448	2008-04-08	existing_value
+449	val_449	2008-04-08	existing_value
+452	val_452	2008-04-08	existing_value
+453	val_453	2008-04-08	existing_value
+454	val_454	2008-04-08	existing_value
+454	val_454	2008-04-08	existing_value
+454	val_454	2008-04-08	existing_value
+455	val_455	2008-04-08	existing_value
+457	val_457	2008-04-08	existing_value
+458	val_458	2008-04-08	existing_value
+458	val_458	2008-04-08	existing_value
+459	val_459	2008-04-08	existing_value
+459	val_459	2008-04-08	existing_value
+460	val_460	2008-04-08	existing_value
+462	val_462	2008-04-08	existing_value
+462	val_462	2008-04-08	existing_value
+463	val_463	2008-04-08	existing_value
+463	val_463	2008-04-08	existing_value
+466	val_466	2008-04-08	existing_value
+466	val_466	2008-04-08	existing_value
+466	val_466	2008-04-08	existing_value
+467	val_467	2008-04-08	existing_value
+468	val_468	2008-04-08	existing_value
+468	val_468	2008-04-08	existing_value
+468	val_468	2008-04-08	existing_value
+468	val_468	2008-04-08	existing_value
+469	val_469	2008-04-08	existing_value
+469	val_469	2008-04-08	existing_value
+469	val_469	2008-04-08	existing_value
+469	val_469	2008-04-08	existing_value
+469	val_469	2008-04-08	existing_value
+470	val_470	2008-04-08	existing_value
+472	val_472	2008-04-08	existing_value
+475	val_475	2008-04-08	existing_value
+477	val_477	2008-04-08	existing_value
+478	val_478	2008-04-08	existing_value
+478	val_478	2008-04-08	existing_value
+479	val_479	2008-04-08	existing_value
+480	val_480	2008-04-08	existing_value
+480	val_480	2008-04-08	existing_value
+480	val_480	2008-04-08	existing_value
+481	val_481	2008-04-08	existing_value
+482	val_482	2008-04-08	existing_value
+483	val_483	2008-04-08	existing_value
+484	val_484	2008-04-08	existing_value
+485	val_485	2008-04-08	existing_value
+487	val_487	2008-04-08	existing_value
+489	val_489	2008-04-08	existing_value
+489	val_489	2008-04-08	existing_value
+489	val_489	2008-04-08	existing_value
+489	val_489	2008-04-08	existing_value
+490	val_490	2008-04-08	existing_value
+491	val_491	2008-04-08	existing_value
+492	val_492	2008-04-08	existing_value
+492	val_492	2008-04-08	existing_value
+493	val_493	2008-04-08	existing_value
+494	val_494	2008-04-08	existing_value
+495	val_495	2008-04-08	existing_value
+496	val_496	2008-04-08	existing_value
+497	val_497	2008-04-08	existing_value
+498	val_498	2008-04-08	existing_value
+498	val_498	2008-04-08	existing_value
+498	val_498	2008-04-08	existing_value
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 b/sql/hive/src/test/resources/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8
deleted file mode 100644
index 3bbd322e374fff9d7538d0d56a18016b6d3a698d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8
+++ /dev/null
@@ -1,1500 +0,0 @@
-238	val_238	2008-04-08	11
-86	val_86	2008-04-08	11
-311	val_311	2008-04-08	11
-27	val_27	2008-04-08	11
-165	val_165	2008-04-08	11
-409	val_409	2008-04-08	11
-255	val_255	2008-04-08	11
-278	val_278	2008-04-08	11
-98	val_98	2008-04-08	11
-484	val_484	2008-04-08	11
-265	val_265	2008-04-08	11
-193	val_193	2008-04-08	11
-401	val_401	2008-04-08	11
-150	val_150	2008-04-08	11
-273	val_273	2008-04-08	11
-224	val_224	2008-04-08	11
-369	val_369	2008-04-08	11
-66	val_66	2008-04-08	11
-128	val_128	2008-04-08	11
-213	val_213	2008-04-08	11
-146	val_146	2008-04-08	11
-406	val_406	2008-04-08	11
-429	val_429	2008-04-08	11
-374	val_374	2008-04-08	11
-152	val_152	2008-04-08	11
-469	val_469	2008-04-08	11
-145	val_145	2008-04-08	11
-495	val_495	2008-04-08	11
-37	val_37	2008-04-08	11
-327	val_327	2008-04-08	11
-281	val_281	2008-04-08	11
-277	val_277	2008-04-08	11
-209	val_209	2008-04-08	11
-15	val_15	2008-04-08	11
-82	val_82	2008-04-08	11
-403	val_403	2008-04-08	11
-166	val_166	2008-04-08	11
-417	val_417	2008-04-08	11
-430	val_430	2008-04-08	11
-252	val_252	2008-04-08	11
-292	val_292	2008-04-08	11
-219	val_219	2008-04-08	11
-287	val_287	2008-04-08	11
-153	val_153	2008-04-08	11
-193	val_193	2008-04-08	11
-338	val_338	2008-04-08	11
-446	val_446	2008-04-08	11
-459	val_459	2008-04-08	11
-394	val_394	2008-04-08	11
-237	val_237	2008-04-08	11
-482	val_482	2008-04-08	11
-174	val_174	2008-04-08	11
-413	val_413	2008-04-08	11
-494	val_494	2008-04-08	11
-207	val_207	2008-04-08	11
-199	val_199	2008-04-08	11
-466	val_466	2008-04-08	11
-208	val_208	2008-04-08	11
-174	val_174	2008-04-08	11
-399	val_399	2008-04-08	11
-396	val_396	2008-04-08	11
-247	val_247	2008-04-08	11
-417	val_417	2008-04-08	11
-489	val_489	2008-04-08	11
-162	val_162	2008-04-08	11
-377	val_377	2008-04-08	11
-397	val_397	2008-04-08	11
-309	val_309	2008-04-08	11
-365	val_365	2008-04-08	11
-266	val_266	2008-04-08	11
-439	val_439	2008-04-08	11
-342	val_342	2008-04-08	11
-367	val_367	2008-04-08	11
-325	val_325	2008-04-08	11
-167	val_167	2008-04-08	11
-195	val_195	2008-04-08	11
-475	val_475	2008-04-08	11
-17	val_17	2008-04-08	11
-113	val_113	2008-04-08	11
-155	val_155	2008-04-08	11
-203	val_203	2008-04-08	11
-339	val_339	2008-04-08	11
-0	val_0	2008-04-08	11
-455	val_455	2008-04-08	11
-128	val_128	2008-04-08	11
-311	val_311	2008-04-08	11
-316	val_316	2008-04-08	11
-57	val_57	2008-04-08	11
-302	val_302	2008-04-08	11
-205	val_205	2008-04-08	11
-149	val_149	2008-04-08	11
-438	val_438	2008-04-08	11
-345	val_345	2008-04-08	11
-129	val_129	2008-04-08	11
-170	val_170	2008-04-08	11
-20	val_20	2008-04-08	11
-489	val_489	2008-04-08	11
-157	val_157	2008-04-08	11
-378	val_378	2008-04-08	11
-221	val_221	2008-04-08	11
-92	val_92	2008-04-08	11
-111	val_111	2008-04-08	11
-47	val_47	2008-04-08	11
-72	val_72	2008-04-08	11
-4	val_4	2008-04-08	11
-280	val_280	2008-04-08	11
-35	val_35	2008-04-08	11
-427	val_427	2008-04-08	11
-277	val_277	2008-04-08	11
-208	val_208	2008-04-08	11
-356	val_356	2008-04-08	11
-399	val_399	2008-04-08	11
-169	val_169	2008-04-08	11
-382	val_382	2008-04-08	11
-498	val_498	2008-04-08	11
-125	val_125	2008-04-08	11
-386	val_386	2008-04-08	11
-437	val_437	2008-04-08	11
-469	val_469	2008-04-08	11
-192	val_192	2008-04-08	11
-286	val_286	2008-04-08	11
-187	val_187	2008-04-08	11
-176	val_176	2008-04-08	11
-54	val_54	2008-04-08	11
-459	val_459	2008-04-08	11
-51	val_51	2008-04-08	11
-138	val_138	2008-04-08	11
-103	val_103	2008-04-08	11
-239	val_239	2008-04-08	11
-213	val_213	2008-04-08	11
-216	val_216	2008-04-08	11
-430	val_430	2008-04-08	11
-278	val_278	2008-04-08	11
-176	val_176	2008-04-08	11
-289	val_289	2008-04-08	11
-221	val_221	2008-04-08	11
-65	val_65	2008-04-08	11
-318	val_318	2008-04-08	11
-332	val_332	2008-04-08	11
-311	val_311	2008-04-08	11
-275	val_275	2008-04-08	11
-137	val_137	2008-04-08	11
-241	val_241	2008-04-08	11
-83	val_83	2008-04-08	11
-333	val_333	2008-04-08	11
-180	val_180	2008-04-08	11
-284	val_284	2008-04-08	11
-12	val_12	2008-04-08	11
-230	val_230	2008-04-08	11
-181	val_181	2008-04-08	11
-67	val_67	2008-04-08	11
-260	val_260	2008-04-08	11
-404	val_404	2008-04-08	11
-384	val_384	2008-04-08	11
-489	val_489	2008-04-08	11
-353	val_353	2008-04-08	11
-373	val_373	2008-04-08	11
-272	val_272	2008-04-08	11
-138	val_138	2008-04-08	11
-217	val_217	2008-04-08	11
-84	val_84	2008-04-08	11
-348	val_348	2008-04-08	11
-466	val_466	2008-04-08	11
-58	val_58	2008-04-08	11
-8	val_8	2008-04-08	11
-411	val_411	2008-04-08	11
-230	val_230	2008-04-08	11
-208	val_208	2008-04-08	11
-348	val_348	2008-04-08	11
-24	val_24	2008-04-08	11
-463	val_463	2008-04-08	11
-431	val_431	2008-04-08	11
-179	val_179	2008-04-08	11
-172	val_172	2008-04-08	11
-42	val_42	2008-04-08	11
-129	val_129	2008-04-08	11
-158	val_158	2008-04-08	11
-119	val_119	2008-04-08	11
-496	val_496	2008-04-08	11
-0	val_0	2008-04-08	11
-322	val_322	2008-04-08	11
-197	val_197	2008-04-08	11
-468	val_468	2008-04-08	11
-393	val_393	2008-04-08	11
-454	val_454	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-199	val_199	2008-04-08	11
-191	val_191	2008-04-08	11
-418	val_418	2008-04-08	11
-96	val_96	2008-04-08	11
-26	val_26	2008-04-08	11
-165	val_165	2008-04-08	11
-327	val_327	2008-04-08	11
-230	val_230	2008-04-08	11
-205	val_205	2008-04-08	11
-120	val_120	2008-04-08	11
-131	val_131	2008-04-08	11
-51	val_51	2008-04-08	11
-404	val_404	2008-04-08	11
-43	val_43	2008-04-08	11
-436	val_436	2008-04-08	11
-156	val_156	2008-04-08	11
-469	val_469	2008-04-08	11
-468	val_468	2008-04-08	11
-308	val_308	2008-04-08	11
-95	val_95	2008-04-08	11
-196	val_196	2008-04-08	11
-288	val_288	2008-04-08	11
-481	val_481	2008-04-08	11
-457	val_457	2008-04-08	11
-98	val_98	2008-04-08	11
-282	val_282	2008-04-08	11
-197	val_197	2008-04-08	11
-187	val_187	2008-04-08	11
-318	val_318	2008-04-08	11
-318	val_318	2008-04-08	11
-409	val_409	2008-04-08	11
-470	val_470	2008-04-08	11
-137	val_137	2008-04-08	11
-369	val_369	2008-04-08	11
-316	val_316	2008-04-08	11
-169	val_169	2008-04-08	11
-413	val_413	2008-04-08	11
-85	val_85	2008-04-08	11
-77	val_77	2008-04-08	11
-0	val_0	2008-04-08	11
-490	val_490	2008-04-08	11
-87	val_87	2008-04-08	11
-364	val_364	2008-04-08	11
-179	val_179	2008-04-08	11
-118	val_118	2008-04-08	11
-134	val_134	2008-04-08	11
-395	val_395	2008-04-08	11
-282	val_282	2008-04-08	11
-138	val_138	2008-04-08	11
-238	val_238	2008-04-08	11
-419	val_419	2008-04-08	11
-15	val_15	2008-04-08	11
-118	val_118	2008-04-08	11
-72	val_72	2008-04-08	11
-90	val_90	2008-04-08	11
-307	val_307	2008-04-08	11
-19	val_19	2008-04-08	11
-435	val_435	2008-04-08	11
-10	val_10	2008-04-08	11
-277	val_277	2008-04-08	11
-273	val_273	2008-04-08	11
-306	val_306	2008-04-08	11
-224	val_224	2008-04-08	11
-309	val_309	2008-04-08	11
-389	val_389	2008-04-08	11
-327	val_327	2008-04-08	11
-242	val_242	2008-04-08	11
-369	val_369	2008-04-08	11
-392	val_392	2008-04-08	11
-272	val_272	2008-04-08	11
-331	val_331	2008-04-08	11
-401	val_401	2008-04-08	11
-242	val_242	2008-04-08	11
-452	val_452	2008-04-08	11
-177	val_177	2008-04-08	11
-226	val_226	2008-04-08	11
-5	val_5	2008-04-08	11
-497	val_497	2008-04-08	11
-402	val_402	2008-04-08	11
-396	val_396	2008-04-08	11
-317	val_317	2008-04-08	11
-395	val_395	2008-04-08	11
-58	val_58	2008-04-08	11
-35	val_35	2008-04-08	11
-336	val_336	2008-04-08	11
-95	val_95	2008-04-08	11
-11	val_11	2008-04-08	11
-168	val_168	2008-04-08	11
-34	val_34	2008-04-08	11
-229	val_229	2008-04-08	11
-233	val_233	2008-04-08	11
-143	val_143	2008-04-08	11
-472	val_472	2008-04-08	11
-322	val_322	2008-04-08	11
-498	val_498	2008-04-08	11
-160	val_160	2008-04-08	11
-195	val_195	2008-04-08	11
-42	val_42	2008-04-08	11
-321	val_321	2008-04-08	11
-430	val_430	2008-04-08	11
-119	val_119	2008-04-08	11
-489	val_489	2008-04-08	11
-458	val_458	2008-04-08	11
-78	val_78	2008-04-08	11
-76	val_76	2008-04-08	11
-41	val_41	2008-04-08	11
-223	val_223	2008-04-08	11
-492	val_492	2008-04-08	11
-149	val_149	2008-04-08	11
-449	val_449	2008-04-08	11
-218	val_218	2008-04-08	11
-228	val_228	2008-04-08	11
-138	val_138	2008-04-08	11
-453	val_453	2008-04-08	11
-30	val_30	2008-04-08	11
-209	val_209	2008-04-08	11
-64	val_64	2008-04-08	11
-468	val_468	2008-04-08	11
-76	val_76	2008-04-08	11
-74	val_74	2008-04-08	11
-342	val_342	2008-04-08	11
-69	val_69	2008-04-08	11
-230	val_230	2008-04-08	11
-33	val_33	2008-04-08	11
-368	val_368	2008-04-08	11
-103	val_103	2008-04-08	11
-296	val_296	2008-04-08	11
-113	val_113	2008-04-08	11
-216	val_216	2008-04-08	11
-367	val_367	2008-04-08	11
-344	val_344	2008-04-08	11
-167	val_167	2008-04-08	11
-274	val_274	2008-04-08	11
-219	val_219	2008-04-08	11
-239	val_239	2008-04-08	11
-485	val_485	2008-04-08	11
-116	val_116	2008-04-08	11
-223	val_223	2008-04-08	11
-256	val_256	2008-04-08	11
-263	val_263	2008-04-08	11
-70	val_70	2008-04-08	11
-487	val_487	2008-04-08	11
-480	val_480	2008-04-08	11
-401	val_401	2008-04-08	11
-288	val_288	2008-04-08	11
-191	val_191	2008-04-08	11
-5	val_5	2008-04-08	11
-244	val_244	2008-04-08	11
-438	val_438	2008-04-08	11
-128	val_128	2008-04-08	11
-467	val_467	2008-04-08	11
-432	val_432	2008-04-08	11
-202	val_202	2008-04-08	11
-316	val_316	2008-04-08	11
-229	val_229	2008-04-08	11
-469	val_469	2008-04-08	11
-463	val_463	2008-04-08	11
-280	val_280	2008-04-08	11
-2	val_2	2008-04-08	11
-35	val_35	2008-04-08	11
-283	val_283	2008-04-08	11
-331	val_331	2008-04-08	11
-235	val_235	2008-04-08	11
-80	val_80	2008-04-08	11
-44	val_44	2008-04-08	11
-193	val_193	2008-04-08	11
-321	val_321	2008-04-08	11
-335	val_335	2008-04-08	11
-104	val_104	2008-04-08	11
-466	val_466	2008-04-08	11
-366	val_366	2008-04-08	11
-175	val_175	2008-04-08	11
-403	val_403	2008-04-08	11
-483	val_483	2008-04-08	11
-53	val_53	2008-04-08	11
-105	val_105	2008-04-08	11
-257	val_257	2008-04-08	11
-406	val_406	2008-04-08	11
-409	val_409	2008-04-08	11
-190	val_190	2008-04-08	11
-406	val_406	2008-04-08	11
-401	val_401	2008-04-08	11
-114	val_114	2008-04-08	11
-258	val_258	2008-04-08	11
-90	val_90	2008-04-08	11
-203	val_203	2008-04-08	11
-262	val_262	2008-04-08	11
-348	val_348	2008-04-08	11
-424	val_424	2008-04-08	11
-12	val_12	2008-04-08	11
-396	val_396	2008-04-08	11
-201	val_201	2008-04-08	11
-217	val_217	2008-04-08	11
-164	val_164	2008-04-08	11
-431	val_431	2008-04-08	11
-454	val_454	2008-04-08	11
-478	val_478	2008-04-08	11
-298	val_298	2008-04-08	11
-125	val_125	2008-04-08	11
-431	val_431	2008-04-08	11
-164	val_164	2008-04-08	11
-424	val_424	2008-04-08	11
-187	val_187	2008-04-08	11
-382	val_382	2008-04-08	11
-5	val_5	2008-04-08	11
-70	val_70	2008-04-08	11
-397	val_397	2008-04-08	11
-480	val_480	2008-04-08	11
-291	val_291	2008-04-08	11
-24	val_24	2008-04-08	11
-351	val_351	2008-04-08	11
-255	val_255	2008-04-08	11
-104	val_104	2008-04-08	11
-70	val_70	2008-04-08	11
-163	val_163	2008-04-08	11
-438	val_438	2008-04-08	11
-119	val_119	2008-04-08	11
-414	val_414	2008-04-08	11
-200	val_200	2008-04-08	11
-491	val_491	2008-04-08	11
-237	val_237	2008-04-08	11
-439	val_439	2008-04-08	11
-360	val_360	2008-04-08	11
-248	val_248	2008-04-08	11
-479	val_479	2008-04-08	11
-305	val_305	2008-04-08	11
-417	val_417	2008-04-08	11
-199	val_199	2008-04-08	11
-444	val_444	2008-04-08	11
-120	val_120	2008-04-08	11
-429	val_429	2008-04-08	11
-169	val_169	2008-04-08	11
-443	val_443	2008-04-08	11
-323	val_323	2008-04-08	11
-325	val_325	2008-04-08	11
-277	val_277	2008-04-08	11
-230	val_230	2008-04-08	11
-478	val_478	2008-04-08	11
-178	val_178	2008-04-08	11
-468	val_468	2008-04-08	11
-310	val_310	2008-04-08	11
-317	val_317	2008-04-08	11
-333	val_333	2008-04-08	11
-493	val_493	2008-04-08	11
-460	val_460	2008-04-08	11
-207	val_207	2008-04-08	11
-249	val_249	2008-04-08	11
-265	val_265	2008-04-08	11
-480	val_480	2008-04-08	11
-83	val_83	2008-04-08	11
-136	val_136	2008-04-08	11
-353	val_353	2008-04-08	11
-172	val_172	2008-04-08	11
-214	val_214	2008-04-08	11
-462	val_462	2008-04-08	11
-233	val_233	2008-04-08	11
-406	val_406	2008-04-08	11
-133	val_133	2008-04-08	11
-175	val_175	2008-04-08	11
-189	val_189	2008-04-08	11
-454	val_454	2008-04-08	11
-375	val_375	2008-04-08	11
-401	val_401	2008-04-08	11
-421	val_421	2008-04-08	11
-407	val_407	2008-04-08	11
-384	val_384	2008-04-08	11
-256	val_256	2008-04-08	11
-26	val_26	2008-04-08	11
-134	val_134	2008-04-08	11
-67	val_67	2008-04-08	11
-384	val_384	2008-04-08	11
-379	val_379	2008-04-08	11
-18	val_18	2008-04-08	11
-462	val_462	2008-04-08	11
-492	val_492	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-9	val_9	2008-04-08	11
-341	val_341	2008-04-08	11
-498	val_498	2008-04-08	11
-146	val_146	2008-04-08	11
-458	val_458	2008-04-08	11
-362	val_362	2008-04-08	11
-186	val_186	2008-04-08	11
-285	val_285	2008-04-08	11
-348	val_348	2008-04-08	11
-167	val_167	2008-04-08	11
-18	val_18	2008-04-08	11
-273	val_273	2008-04-08	11
-183	val_183	2008-04-08	11
-281	val_281	2008-04-08	11
-344	val_344	2008-04-08	11
-97	val_97	2008-04-08	11
-469	val_469	2008-04-08	11
-315	val_315	2008-04-08	11
-84	val_84	2008-04-08	11
-28	val_28	2008-04-08	11
-37	val_37	2008-04-08	11
-448	val_448	2008-04-08	11
-152	val_152	2008-04-08	11
-348	val_348	2008-04-08	11
-307	val_307	2008-04-08	11
-194	val_194	2008-04-08	11
-414	val_414	2008-04-08	11
-477	val_477	2008-04-08	11
-222	val_222	2008-04-08	11
-126	val_126	2008-04-08	11
-90	val_90	2008-04-08	11
-169	val_169	2008-04-08	11
-403	val_403	2008-04-08	11
-400	val_400	2008-04-08	11
-200	val_200	2008-04-08	11
-97	val_97	2008-04-08	11
-238	val_238	2008-04-08	12
-86	val_86	2008-04-08	12
-311	val_311	2008-04-08	12
-27	val_27	2008-04-08	12
-165	val_165	2008-04-08	12
-409	val_409	2008-04-08	12
-255	val_255	2008-04-08	12
-278	val_278	2008-04-08	12
-98	val_98	2008-04-08	12
-484	val_484	2008-04-08	12
-265	val_265	2008-04-08	12
-193	val_193	2008-04-08	12
-401	val_401	2008-04-08	12
-150	val_150	2008-04-08	12
-273	val_273	2008-04-08	12
-224	val_224	2008-04-08	12
-369	val_369	2008-04-08	12
-66	val_66	2008-04-08	12
-128	val_128	2008-04-08	12
-213	val_213	2008-04-08	12
-146	val_146	2008-04-08	12
-406	val_406	2008-04-08	12
-429	val_429	2008-04-08	12
-374	val_374	2008-04-08	12
-152	val_152	2008-04-08	12
-469	val_469	2008-04-08	12
-145	val_145	2008-04-08	12
-495	val_495	2008-04-08	12
-37	val_37	2008-04-08	12
-327	val_327	2008-04-08	12
-281	val_281	2008-04-08	12
-277	val_277	2008-04-08	12
-209	val_209	2008-04-08	12
-15	val_15	2008-04-08	12
-82	val_82	2008-04-08	12
-403	val_403	2008-04-08	12
-166	val_166	2008-04-08	12
-417	val_417	2008-04-08	12
-430	val_430	2008-04-08	12
-252	val_252	2008-04-08	12
-292	val_292	2008-04-08	12
-219	val_219	2008-04-08	12
-287	val_287	2008-04-08	12
-153	val_153	2008-04-08	12
-193	val_193	2008-04-08	12
-338	val_338	2008-04-08	12
-446	val_446	2008-04-08	12
-459	val_459	2008-04-08	12
-394	val_394	2008-04-08	12
-237	val_237	2008-04-08	12
-482	val_482	2008-04-08	12
-174	val_174	2008-04-08	12
-413	val_413	2008-04-08	12
-494	val_494	2008-04-08	12
-207	val_207	2008-04-08	12
-199	val_199	2008-04-08	12
-466	val_466	2008-04-08	12
-208	val_208	2008-04-08	12
-174	val_174	2008-04-08	12
-399	val_399	2008-04-08	12
-396	val_396	2008-04-08	12
-247	val_247	2008-04-08	12
-417	val_417	2008-04-08	12
-489	val_489	2008-04-08	12
-162	val_162	2008-04-08	12
-377	val_377	2008-04-08	12
-397	val_397	2008-04-08	12
-309	val_309	2008-04-08	12
-365	val_365	2008-04-08	12
-266	val_266	2008-04-08	12
-439	val_439	2008-04-08	12
-342	val_342	2008-04-08	12
-367	val_367	2008-04-08	12
-325	val_325	2008-04-08	12
-167	val_167	2008-04-08	12
-195	val_195	2008-04-08	12
-475	val_475	2008-04-08	12
-17	val_17	2008-04-08	12
-113	val_113	2008-04-08	12
-155	val_155	2008-04-08	12
-203	val_203	2008-04-08	12
-339	val_339	2008-04-08	12
-0	val_0	2008-04-08	12
-455	val_455	2008-04-08	12
-128	val_128	2008-04-08	12
-311	val_311	2008-04-08	12
-316	val_316	2008-04-08	12
-57	val_57	2008-04-08	12
-302	val_302	2008-04-08	12
-205	val_205	2008-04-08	12
-149	val_149	2008-04-08	12
-438	val_438	2008-04-08	12
-345	val_345	2008-04-08	12
-129	val_129	2008-04-08	12
-170	val_170	2008-04-08	12
-20	val_20	2008-04-08	12
-489	val_489	2008-04-08	12
-157	val_157	2008-04-08	12
-378	val_378	2008-04-08	12
-221	val_221	2008-04-08	12
-92	val_92	2008-04-08	12
-111	val_111	2008-04-08	12
-47	val_47	2008-04-08	12
-72	val_72	2008-04-08	12
-4	val_4	2008-04-08	12
-280	val_280	2008-04-08	12
-35	val_35	2008-04-08	12
-427	val_427	2008-04-08	12
-277	val_277	2008-04-08	12
-208	val_208	2008-04-08	12
-356	val_356	2008-04-08	12
-399	val_399	2008-04-08	12
-169	val_169	2008-04-08	12
-382	val_382	2008-04-08	12
-498	val_498	2008-04-08	12
-125	val_125	2008-04-08	12
-386	val_386	2008-04-08	12
-437	val_437	2008-04-08	12
-469	val_469	2008-04-08	12
-192	val_192	2008-04-08	12
-286	val_286	2008-04-08	12
-187	val_187	2008-04-08	12
-176	val_176	2008-04-08	12
-54	val_54	2008-04-08	12
-459	val_459	2008-04-08	12
-51	val_51	2008-04-08	12
-138	val_138	2008-04-08	12
-103	val_103	2008-04-08	12
-239	val_239	2008-04-08	12
-213	val_213	2008-04-08	12
-216	val_216	2008-04-08	12
-430	val_430	2008-04-08	12
-278	val_278	2008-04-08	12
-176	val_176	2008-04-08	12
-289	val_289	2008-04-08	12
-221	val_221	2008-04-08	12
-65	val_65	2008-04-08	12
-318	val_318	2008-04-08	12
-332	val_332	2008-04-08	12
-311	val_311	2008-04-08	12
-275	val_275	2008-04-08	12
-137	val_137	2008-04-08	12
-241	val_241	2008-04-08	12
-83	val_83	2008-04-08	12
-333	val_333	2008-04-08	12
-180	val_180	2008-04-08	12
-284	val_284	2008-04-08	12
-12	val_12	2008-04-08	12
-230	val_230	2008-04-08	12
-181	val_181	2008-04-08	12
-67	val_67	2008-04-08	12
-260	val_260	2008-04-08	12
-404	val_404	2008-04-08	12
-384	val_384	2008-04-08	12
-489	val_489	2008-04-08	12
-353	val_353	2008-04-08	12
-373	val_373	2008-04-08	12
-272	val_272	2008-04-08	12
-138	val_138	2008-04-08	12
-217	val_217	2008-04-08	12
-84	val_84	2008-04-08	12
-348	val_348	2008-04-08	12
-466	val_466	2008-04-08	12
-58	val_58	2008-04-08	12
-8	val_8	2008-04-08	12
-411	val_411	2008-04-08	12
-230	val_230	2008-04-08	12
-208	val_208	2008-04-08	12
-348	val_348	2008-04-08	12
-24	val_24	2008-04-08	12
-463	val_463	2008-04-08	12
-431	val_431	2008-04-08	12
-179	val_179	2008-04-08	12
-172	val_172	2008-04-08	12
-42	val_42	2008-04-08	12
-129	val_129	2008-04-08	12
-158	val_158	2008-04-08	12
-119	val_119	2008-04-08	12
-496	val_496	2008-04-08	12
-0	val_0	2008-04-08	12
-322	val_322	2008-04-08	12
-197	val_197	2008-04-08	12
-468	val_468	2008-04-08	12
-393	val_393	2008-04-08	12
-454	val_454	2008-04-08	12
-100	val_100	2008-04-08	12
-298	val_298	2008-04-08	12
-199	val_199	2008-04-08	12
-191	val_191	2008-04-08	12
-418	val_418	2008-04-08	12
-96	val_96	2008-04-08	12
-26	val_26	2008-04-08	12
-165	val_165	2008-04-08	12
-327	val_327	2008-04-08	12
-230	val_230	2008-04-08	12
-205	val_205	2008-04-08	12
-120	val_120	2008-04-08	12
-131	val_131	2008-04-08	12
-51	val_51	2008-04-08	12
-404	val_404	2008-04-08	12
-43	val_43	2008-04-08	12
-436	val_436	2008-04-08	12
-156	val_156	2008-04-08	12
-469	val_469	2008-04-08	12
-468	val_468	2008-04-08	12
-308	val_308	2008-04-08	12
-95	val_95	2008-04-08	12
-196	val_196	2008-04-08	12
-288	val_288	2008-04-08	12
-481	val_481	2008-04-08	12
-457	val_457	2008-04-08	12
-98	val_98	2008-04-08	12
-282	val_282	2008-04-08	12
-197	val_197	2008-04-08	12
-187	val_187	2008-04-08	12
-318	val_318	2008-04-08	12
-318	val_318	2008-04-08	12
-409	val_409	2008-04-08	12
-470	val_470	2008-04-08	12
-137	val_137	2008-04-08	12
-369	val_369	2008-04-08	12
-316	val_316	2008-04-08	12
-169	val_169	2008-04-08	12
-413	val_413	2008-04-08	12
-85	val_85	2008-04-08	12
-77	val_77	2008-04-08	12
-0	val_0	2008-04-08	12
-490	val_490	2008-04-08	12
-87	val_87	2008-04-08	12
-364	val_364	2008-04-08	12
-179	val_179	2008-04-08	12
-118	val_118	2008-04-08	12
-134	val_134	2008-04-08	12
-395	val_395	2008-04-08	12
-282	val_282	2008-04-08	12
-138	val_138	2008-04-08	12
-238	val_238	2008-04-08	12
-419	val_419	2008-04-08	12
-15	val_15	2008-04-08	12
-118	val_118	2008-04-08	12
-72	val_72	2008-04-08	12
-90	val_90	2008-04-08	12
-307	val_307	2008-04-08	12
-19	val_19	2008-04-08	12
-435	val_435	2008-04-08	12
-10	val_10	2008-04-08	12
-277	val_277	2008-04-08	12
-273	val_273	2008-04-08	12
-306	val_306	2008-04-08	12
-224	val_224	2008-04-08	12
-309	val_309	2008-04-08	12
-389	val_389	2008-04-08	12
-327	val_327	2008-04-08	12
-242	val_242	2008-04-08	12
-369	val_369	2008-04-08	12
-392	val_392	2008-04-08	12
-272	val_272	2008-04-08	12
-331	val_331	2008-04-08	12
-401	val_401	2008-04-08	12
-242	val_242	2008-04-08	12
-452	val_452	2008-04-08	12
-177	val_177	2008-04-08	12
-226	val_226	2008-04-08	12
-5	val_5	2008-04-08	12
-497	val_497	2008-04-08	12
-402	val_402	2008-04-08	12
-396	val_396	2008-04-08	12
-317	val_317	2008-04-08	12
-395	val_395	2008-04-08	12
-58	val_58	2008-04-08	12
-35	val_35	2008-04-08	12
-336	val_336	2008-04-08	12
-95	val_95	2008-04-08	12
-11	val_11	2008-04-08	12
-168	val_168	2008-04-08	12
-34	val_34	2008-04-08	12
-229	val_229	2008-04-08	12
-233	val_233	2008-04-08	12
-143	val_143	2008-04-08	12
-472	val_472	2008-04-08	12
-322	val_322	2008-04-08	12
-498	val_498	2008-04-08	12
-160	val_160	2008-04-08	12
-195	val_195	2008-04-08	12
-42	val_42	2008-04-08	12
-321	val_321	2008-04-08	12
-430	val_430	2008-04-08	12
-119	val_119	2008-04-08	12
-489	val_489	2008-04-08	12
-458	val_458	2008-04-08	12
-78	val_78	2008-04-08	12
-76	val_76	2008-04-08	12
-41	val_41	2008-04-08	12
-223	val_223	2008-04-08	12
-492	val_492	2008-04-08	12
-149	val_149	2008-04-08	12
-449	val_449	2008-04-08	12
-218	val_218	2008-04-08	12
-228	val_228	2008-04-08	12
-138	val_138	2008-04-08	12
-453	val_453	2008-04-08	12
-30	val_30	2008-04-08	12
-209	val_209	2008-04-08	12
-64	val_64	2008-04-08	12
-468	val_468	2008-04-08	12
-76	val_76	2008-04-08	12
-74	val_74	2008-04-08	12
-342	val_342	2008-04-08	12
-69	val_69	2008-04-08	12
-230	val_230	2008-04-08	12
-33	val_33	2008-04-08	12
-368	val_368	2008-04-08	12
-103	val_103	2008-04-08	12
-296	val_296	2008-04-08	12
-113	val_113	2008-04-08	12
-216	val_216	2008-04-08	12
-367	val_367	2008-04-08	12
-344	val_344	2008-04-08	12
-167	val_167	2008-04-08	12
-274	val_274	2008-04-08	12
-219	val_219	2008-04-08	12
-239	val_239	2008-04-08	12
-485	val_485	2008-04-08	12
-116	val_116	2008-04-08	12
-223	val_223	2008-04-08	12
-256	val_256	2008-04-08	12
-263	val_263	2008-04-08	12
-70	val_70	2008-04-08	12
-487	val_487	2008-04-08	12
-480	val_480	2008-04-08	12
-401	val_401	2008-04-08	12
-288	val_288	2008-04-08	12
-191	val_191	2008-04-08	12
-5	val_5	2008-04-08	12
-244	val_244	2008-04-08	12
-438	val_438	2008-04-08	12
-128	val_128	2008-04-08	12
-467	val_467	2008-04-08	12
-432	val_432	2008-04-08	12
-202	val_202	2008-04-08	12
-316	val_316	2008-04-08	12
-229	val_229	2008-04-08	12
-469	val_469	2008-04-08	12
-463	val_463	2008-04-08	12
-280	val_280	2008-04-08	12
-2	val_2	2008-04-08	12
-35	val_35	2008-04-08	12
-283	val_283	2008-04-08	12
-331	val_331	2008-04-08	12
-235	val_235	2008-04-08	12
-80	val_80	2008-04-08	12
-44	val_44	2008-04-08	12
-193	val_193	2008-04-08	12
-321	val_321	2008-04-08	12
-335	val_335	2008-04-08	12
-104	val_104	2008-04-08	12
-466	val_466	2008-04-08	12
-366	val_366	2008-04-08	12
-175	val_175	2008-04-08	12
-403	val_403	2008-04-08	12
-483	val_483	2008-04-08	12
-53	val_53	2008-04-08	12
-105	val_105	2008-04-08	12
-257	val_257	2008-04-08	12
-406	val_406	2008-04-08	12
-409	val_409	2008-04-08	12
-190	val_190	2008-04-08	12
-406	val_406	2008-04-08	12
-401	val_401	2008-04-08	12
-114	val_114	2008-04-08	12
-258	val_258	2008-04-08	12
-90	val_90	2008-04-08	12
-203	val_203	2008-04-08	12
-262	val_262	2008-04-08	12
-348	val_348	2008-04-08	12
-424	val_424	2008-04-08	12
-12	val_12	2008-04-08	12
-396	val_396	2008-04-08	12
-201	val_201	2008-04-08	12
-217	val_217	2008-04-08	12
-164	val_164	2008-04-08	12
-431	val_431	2008-04-08	12
-454	val_454	2008-04-08	12
-478	val_478	2008-04-08	12
-298	val_298	2008-04-08	12
-125	val_125	2008-04-08	12
-431	val_431	2008-04-08	12
-164	val_164	2008-04-08	12
-424	val_424	2008-04-08	12
-187	val_187	2008-04-08	12
-382	val_382	2008-04-08	12
-5	val_5	2008-04-08	12
-70	val_70	2008-04-08	12
-397	val_397	2008-04-08	12
-480	val_480	2008-04-08	12
-291	val_291	2008-04-08	12
-24	val_24	2008-04-08	12
-351	val_351	2008-04-08	12
-255	val_255	2008-04-08	12
-104	val_104	2008-04-08	12
-70	val_70	2008-04-08	12
-163	val_163	2008-04-08	12
-438	val_438	2008-04-08	12
-119	val_119	2008-04-08	12
-414	val_414	2008-04-08	12
-200	val_200	2008-04-08	12
-491	val_491	2008-04-08	12
-237	val_237	2008-04-08	12
-439	val_439	2008-04-08	12
-360	val_360	2008-04-08	12
-248	val_248	2008-04-08	12
-479	val_479	2008-04-08	12
-305	val_305	2008-04-08	12
-417	val_417	2008-04-08	12
-199	val_199	2008-04-08	12
-444	val_444	2008-04-08	12
-120	val_120	2008-04-08	12
-429	val_429	2008-04-08	12
-169	val_169	2008-04-08	12
-443	val_443	2008-04-08	12
-323	val_323	2008-04-08	12
-325	val_325	2008-04-08	12
-277	val_277	2008-04-08	12
-230	val_230	2008-04-08	12
-478	val_478	2008-04-08	12
-178	val_178	2008-04-08	12
-468	val_468	2008-04-08	12
-310	val_310	2008-04-08	12
-317	val_317	2008-04-08	12
-333	val_333	2008-04-08	12
-493	val_493	2008-04-08	12
-460	val_460	2008-04-08	12
-207	val_207	2008-04-08	12
-249	val_249	2008-04-08	12
-265	val_265	2008-04-08	12
-480	val_480	2008-04-08	12
-83	val_83	2008-04-08	12
-136	val_136	2008-04-08	12
-353	val_353	2008-04-08	12
-172	val_172	2008-04-08	12
-214	val_214	2008-04-08	12
-462	val_462	2008-04-08	12
-233	val_233	2008-04-08	12
-406	val_406	2008-04-08	12
-133	val_133	2008-04-08	12
-175	val_175	2008-04-08	12
-189	val_189	2008-04-08	12
-454	val_454	2008-04-08	12
-375	val_375	2008-04-08	12
-401	val_401	2008-04-08	12
-421	val_421	2008-04-08	12
-407	val_407	2008-04-08	12
-384	val_384	2008-04-08	12
-256	val_256	2008-04-08	12
-26	val_26	2008-04-08	12
-134	val_134	2008-04-08	12
-67	val_67	2008-04-08	12
-384	val_384	2008-04-08	12
-379	val_379	2008-04-08	12
-18	val_18	2008-04-08	12
-462	val_462	2008-04-08	12
-492	val_492	2008-04-08	12
-100	val_100	2008-04-08	12
-298	val_298	2008-04-08	12
-9	val_9	2008-04-08	12
-341	val_341	2008-04-08	12
-498	val_498	2008-04-08	12
-146	val_146	2008-04-08	12
-458	val_458	2008-04-08	12
-362	val_362	2008-04-08	12
-186	val_186	2008-04-08	12
-285	val_285	2008-04-08	12
-348	val_348	2008-04-08	12
-167	val_167	2008-04-08	12
-18	val_18	2008-04-08	12
-273	val_273	2008-04-08	12
-183	val_183	2008-04-08	12
-281	val_281	2008-04-08	12
-344	val_344	2008-04-08	12
-97	val_97	2008-04-08	12
-469	val_469	2008-04-08	12
-315	val_315	2008-04-08	12
-84	val_84	2008-04-08	12
-28	val_28	2008-04-08	12
-37	val_37	2008-04-08	12
-448	val_448	2008-04-08	12
-152	val_152	2008-04-08	12
-348	val_348	2008-04-08	12
-307	val_307	2008-04-08	12
-194	val_194	2008-04-08	12
-414	val_414	2008-04-08	12
-477	val_477	2008-04-08	12
-222	val_222	2008-04-08	12
-126	val_126	2008-04-08	12
-90	val_90	2008-04-08	12
-169	val_169	2008-04-08	12
-403	val_403	2008-04-08	12
-400	val_400	2008-04-08	12
-200	val_200	2008-04-08	12
-97	val_97	2008-04-08	12
-238	val_238	2008-04-08	existing_value
-86	val_86	2008-04-08	existing_value
-311	val_311	2008-04-08	existing_value
-27	val_27	2008-04-08	existing_value
-165	val_165	2008-04-08	existing_value
-409	val_409	2008-04-08	existing_value
-255	val_255	2008-04-08	existing_value
-278	val_278	2008-04-08	existing_value
-98	val_98	2008-04-08	existing_value
-484	val_484	2008-04-08	existing_value
-265	val_265	2008-04-08	existing_value
-193	val_193	2008-04-08	existing_value
-401	val_401	2008-04-08	existing_value
-150	val_150	2008-04-08	existing_value
-273	val_273	2008-04-08	existing_value
-224	val_224	2008-04-08	existing_value
-369	val_369	2008-04-08	existing_value
-66	val_66	2008-04-08	existing_value
-128	val_128	2008-04-08	existing_value
-213	val_213	2008-04-08	existing_value
-146	val_146	2008-04-08	existing_value
-406	val_406	2008-04-08	existing_value
-429	val_429	2008-04-08	existing_value
-374	val_374	2008-04-08	existing_value
-152	val_152	2008-04-08	existing_value
-469	val_469	2008-04-08	existing_value
-145	val_145	2008-04-08	existing_value
-495	val_495	2008-04-08	existing_value
-37	val_37	2008-04-08	existing_value
-327	val_327	2008-04-08	existing_value
-281	val_281	2008-04-08	existing_value
-277	val_277	2008-04-08	existing_value
-209	val_209	2008-04-08	existing_value
-15	val_15	2008-04-08	existing_value
-82	val_82	2008-04-08	existing_value
-403	val_403	2008-04-08	existing_value
-166	val_166	2008-04-08	existing_value
-417	val_417	2008-04-08	existing_value
-430	val_430	2008-04-08	existing_value
-252	val_252	2008-04-08	existing_value
-292	val_292	2008-04-08	existing_value
-219	val_219	2008-04-08	existing_value
-287	val_287	2008-04-08	existing_value
-153	val_153	2008-04-08	existing_value
-193	val_193	2008-04-08	existing_value
-338	val_338	2008-04-08	existing_value
-446	val_446	2008-04-08	existing_value
-459	val_459	2008-04-08	existing_value
-394	val_394	2008-04-08	existing_value
-237	val_237	2008-04-08	existing_value
-482	val_482	2008-04-08	existing_value
-174	val_174	2008-04-08	existing_value
-413	val_413	2008-04-08	existing_value
-494	val_494	2008-04-08	existing_value
-207	val_207	2008-04-08	existing_value
-199	val_199	2008-04-08	existing_value
-466	val_466	2008-04-08	existing_value
-208	val_208	2008-04-08	existing_value
-174	val_174	2008-04-08	existing_value
-399	val_399	2008-04-08	existing_value
-396	val_396	2008-04-08	existing_value
-247	val_247	2008-04-08	existing_value
-417	val_417	2008-04-08	existing_value
-489	val_489	2008-04-08	existing_value
-162	val_162	2008-04-08	existing_value
-377	val_377	2008-04-08	existing_value
-397	val_397	2008-04-08	existing_value
-309	val_309	2008-04-08	existing_value
-365	val_365	2008-04-08	existing_value
-266	val_266	2008-04-08	existing_value
-439	val_439	2008-04-08	existing_value
-342	val_342	2008-04-08	existing_value
-367	val_367	2008-04-08	existing_value
-325	val_325	2008-04-08	existing_value
-167	val_167	2008-04-08	existing_value
-195	val_195	2008-04-08	existing_value
-475	val_475	2008-04-08	existing_value
-17	val_17	2008-04-08	existing_value
-113	val_113	2008-04-08	existing_value
-155	val_155	2008-04-08	existing_value
-203	val_203	2008-04-08	existing_value
-339	val_339	2008-04-08	existing_value
-0	val_0	2008-04-08	existing_value
-455	val_455	2008-04-08	existing_value
-128	val_128	2008-04-08	existing_value
-311	val_311	2008-04-08	existing_value
-316	val_316	2008-04-08	existing_value
-57	val_57	2008-04-08	existing_value
-302	val_302	2008-04-08	existing_value
-205	val_205	2008-04-08	existing_value
-149	val_149	2008-04-08	existing_value
-438	val_438	2008-04-08	existing_value
-345	val_345	2008-04-08	existing_value
-129	val_129	2008-04-08	existing_value
-170	val_170	2008-04-08	existing_value
-20	val_20	2008-04-08	existing_value
-489	val_489	2008-04-08	existing_value
-157	val_157	2008-04-08	existing_value
-378	val_378	2008-04-08	existing_value
-221	val_221	2008-04-08	existing_value
-92	val_92	2008-04-08	existing_value
-111	val_111	2008-04-08	existing_value
-47	val_47	2008-04-08	existing_value
-72	val_72	2008-04-08	existing_value
-4	val_4	2008-04-08	existing_value
-280	val_280	2008-04-08	existing_value
-35	val_35	2008-04-08	existing_value
-427	val_427	2008-04-08	existing_value
-277	val_277	2008-04-08	existing_value
-208	val_208	2008-04-08	existing_value
-356	val_356	2008-04-08	existing_value
-399	val_399	2008-04-08	existing_value
-169	val_169	2008-04-08	existing_value
-382	val_382	2008-04-08	existing_value
-498	val_498	2008-04-08	existing_value
-125	val_125	2008-04-08	existing_value
-386	val_386	2008-04-08	existing_value
-437	val_437	2008-04-08	existing_value
-469	val_469	2008-04-08	existing_value
-192	val_192	2008-04-08	existing_value
-286	val_286	2008-04-08	existing_value
-187	val_187	2008-04-08	existing_value
-176	val_176	2008-04-08	existing_value
-54	val_54	2008-04-08	existing_value
-459	val_459	2008-04-08	existing_value
-51	val_51	2008-04-08	existing_value
-138	val_138	2008-04-08	existing_value
-103	val_103	2008-04-08	existing_value
-239	val_239	2008-04-08	existing_value
-213	val_213	2008-04-08	existing_value
-216	val_216	2008-04-08	existing_value
-430	val_430	2008-04-08	existing_value
-278	val_278	2008-04-08	existing_value
-176	val_176	2008-04-08	existing_value
-289	val_289	2008-04-08	existing_value
-221	val_221	2008-04-08	existing_value
-65	val_65	2008-04-08	existing_value
-318	val_318	2008-04-08	existing_value
-332	val_332	2008-04-08	existing_value
-311	val_311	2008-04-08	existing_value
-275	val_275	2008-04-08	existing_value
-137	val_137	2008-04-08	existing_value
-241	val_241	2008-04-08	existing_value
-83	val_83	2008-04-08	existing_value
-333	val_333	2008-04-08	existing_value
-180	val_180	2008-04-08	existing_value
-284	val_284	2008-04-08	existing_value
-12	val_12	2008-04-08	existing_value
-230	val_230	2008-04-08	existing_value
-181	val_181	2008-04-08	existing_value
-67	val_67	2008-04-08	existing_value
-260	val_260	2008-04-08	existing_value
-404	val_404	2008-04-08	existing_value
-384	val_384	2008-04-08	existing_value
-489	val_489	2008-04-08	existing_value
-353	val_353	2008-04-08	existing_value
-373	val_373	2008-04-08	existing_value
-272	val_272	2008-04-08	existing_value
-138	val_138	2008-04-08	existing_value
-217	val_217	2008-04-08	existing_value
-84	val_84	2008-04-08	existing_value
-348	val_348	2008-04-08	existing_value
-466	val_466	2008-04-08	existing_value
-58	val_58	2008-04-08	existing_value
-8	val_8	2008-04-08	existing_value
-411	val_411	2008-04-08	existing_value
-230	val_230	2008-04-08	existing_value
-208	val_208	2008-04-08	existing_value
-348	val_348	2008-04-08	existing_value
-24	val_24	2008-04-08	existing_value
-463	val_463	2008-04-08	existing_value
-431	val_431	2008-04-08	existing_value
-179	val_179	2008-04-08	existing_value
-172	val_172	2008-04-08	existing_value
-42	val_42	2008-04-08	existing_value
-129	val_129	2008-04-08	existing_value
-158	val_158	2008-04-08	existing_value
-119	val_119	2008-04-08	existing_value
-496	val_496	2008-04-08	existing_value
-0	val_0	2008-04-08	existing_value
-322	val_322	2008-04-08	existing_value
-197	val_197	2008-04-08	existing_value
-468	val_468	2008-04-08	existing_value
-393	val_393	2008-04-08	existing_value
-454	val_454	2008-04-08	existing_value
-100	val_100	2008-04-08	existing_value
-298	val_298	2008-04-08	existing_value
-199	val_199	2008-04-08	existing_value
-191	val_191	2008-04-08	existing_value
-418	val_418	2008-04-08	existing_value
-96	val_96	2008-04-08	existing_value
-26	val_26	2008-04-08	existing_value
-165	val_165	2008-04-08	existing_value
-327	val_327	2008-04-08	existing_value
-230	val_230	2008-04-08	existing_value
-205	val_205	2008-04-08	existing_value
-120	val_120	2008-04-08	existing_value
-131	val_131	2008-04-08	existing_value
-51	val_51	2008-04-08	existing_value
-404	val_404	2008-04-08	existing_value
-43	val_43	2008-04-08	existing_value
-436	val_436	2008-04-08	existing_value
-156	val_156	2008-04-08	existing_value
-469	val_469	2008-04-08	existing_value
-468	val_468	2008-04-08	existing_value
-308	val_308	2008-04-08	existing_value
-95	val_95	2008-04-08	existing_value
-196	val_196	2008-04-08	existing_value
-288	val_288	2008-04-08	existing_value
-481	val_481	2008-04-08	existing_value
-457	val_457	2008-04-08	existing_value
-98	val_98	2008-04-08	existing_value
-282	val_282	2008-04-08	existing_value
-197	val_197	2008-04-08	existing_value
-187	val_187	2008-04-08	existing_value
-318	val_318	2008-04-08	existing_value
-318	val_318	2008-04-08	existing_value
-409	val_409	2008-04-08	existing_value
-470	val_470	2008-04-08	existing_value
-137	val_137	2008-04-08	existing_value
-369	val_369	2008-04-08	existing_value
-316	val_316	2008-04-08	existing_value
-169	val_169	2008-04-08	existing_value
-413	val_413	2008-04-08	existing_value
-85	val_85	2008-04-08	existing_value
-77	val_77	2008-04-08	existing_value
-0	val_0	2008-04-08	existing_value
-490	val_490	2008-04-08	existing_value
-87	val_87	2008-04-08	existing_value
-364	val_364	2008-04-08	existing_value
-179	val_179	2008-04-08	existing_value
-118	val_118	2008-04-08	existing_value
-134	val_134	2008-04-08	existing_value
-395	val_395	2008-04-08	existing_value
-282	val_282	2008-04-08	existing_value
-138	val_138	2008-04-08	existing_value
-238	val_238	2008-04-08	existing_value
-419	val_419	2008-04-08	existing_value
-15	val_15	2008-04-08	existing_value
-118	val_118	2008-04-08	existing_value
-72	val_72	2008-04-08	existing_value
-90	val_90	2008-04-08	existing_value
-307	val_307	2008-04-08	existing_value
-19	val_19	2008-04-08	existing_value
-435	val_435	2008-04-08	existing_value
-10	val_10	2008-04-08	existing_value
-277	val_277	2008-04-08	existing_value
-273	val_273	2008-04-08	existing_value
-306	val_306	2008-04-08	existing_value
-224	val_224	2008-04-08	existing_value
-309	val_309	2008-04-08	existing_value
-389	val_389	2008-04-08	existing_value
-327	val_327	2008-04-08	existing_value
-242	val_242	2008-04-08	existing_value
-369	val_369	2008-04-08	existing_value
-392	val_392	2008-04-08	existing_value
-272	val_272	2008-04-08	existing_value
-331	val_331	2008-04-08	existing_value
-401	val_401	2008-04-08	existing_value
-242	val_242	2008-04-08	existing_value
-452	val_452	2008-04-08	existing_value
-177	val_177	2008-04-08	existing_value
-226	val_226	2008-04-08	existing_value
-5	val_5	2008-04-08	existing_value
-497	val_497	2008-04-08	existing_value
-402	val_402	2008-04-08	existing_value
-396	val_396	2008-04-08	existing_value
-317	val_317	2008-04-08	existing_value
-395	val_395	2008-04-08	existing_value
-58	val_58	2008-04-08	existing_value
-35	val_35	2008-04-08	existing_value
-336	val_336	2008-04-08	existing_value
-95	val_95	2008-04-08	existing_value
-11	val_11	2008-04-08	existing_value
-168	val_168	2008-04-08	existing_value
-34	val_34	2008-04-08	existing_value
-229	val_229	2008-04-08	existing_value
-233	val_233	2008-04-08	existing_value
-143	val_143	2008-04-08	existing_value
-472	val_472	2008-04-08	existing_value
-322	val_322	2008-04-08	existing_value
-498	val_498	2008-04-08	existing_value
-160	val_160	2008-04-08	existing_value
-195	val_195	2008-04-08	existing_value
-42	val_42	2008-04-08	existing_value
-321	val_321	2008-04-08	existing_value
-430	val_430	2008-04-08	existing_value
-119	val_119	2008-04-08	existing_value
-489	val_489	2008-04-08	existing_value
-458	val_458	2008-04-08	existing_value
-78	val_78	2008-04-08	existing_value
-76	val_76	2008-04-08	existing_value
-41	val_41	2008-04-08	existing_value
-223	val_223	2008-04-08	existing_value
-492	val_492	2008-04-08	existing_value
-149	val_149	2008-04-08	existing_value
-449	val_449	2008-04-08	existing_value
-218	val_218	2008-04-08	existing_value
-228	val_228	2008-04-08	existing_value
-138	val_138	2008-04-08	existing_value
-453	val_453	2008-04-08	existing_value
-30	val_30	2008-04-08	existing_value
-209	val_209	2008-04-08	existing_value
-64	val_64	2008-04-08	existing_value
-468	val_468	2008-04-08	existing_value
-76	val_76	2008-04-08	existing_value
-74	val_74	2008-04-08	existing_value
-342	val_342	2008-04-08	existing_value
-69	val_69	2008-04-08	existing_value
-230	val_230	2008-04-08	existing_value
-33	val_33	2008-04-08	existing_value
-368	val_368	2008-04-08	existing_value
-103	val_103	2008-04-08	existing_value
-296	val_296	2008-04-08	existing_value
-113	val_113	2008-04-08	existing_value
-216	val_216	2008-04-08	existing_value
-367	val_367	2008-04-08	existing_value
-344	val_344	2008-04-08	existing_value
-167	val_167	2008-04-08	existing_value
-274	val_274	2008-04-08	existing_value
-219	val_219	2008-04-08	existing_value
-239	val_239	2008-04-08	existing_value
-485	val_485	2008-04-08	existing_value
-116	val_116	2008-04-08	existing_value
-223	val_223	2008-04-08	existing_value
-256	val_256	2008-04-08	existing_value
-263	val_263	2008-04-08	existing_value
-70	val_70	2008-04-08	existing_value
-487	val_487	2008-04-08	existing_value
-480	val_480	2008-04-08	existing_value
-401	val_401	2008-04-08	existing_value
-288	val_288	2008-04-08	existing_value
-191	val_191	2008-04-08	existing_value
-5	val_5	2008-04-08	existing_value
-244	val_244	2008-04-08	existing_value
-438	val_438	2008-04-08	existing_value
-128	val_128	2008-04-08	existing_value
-467	val_467	2008-04-08	existing_value
-432	val_432	2008-04-08	existing_value
-202	val_202	2008-04-08	existing_value
-316	val_316	2008-04-08	existing_value
-229	val_229	2008-04-08	existing_value
-469	val_469	2008-04-08	existing_value
-463	val_463	2008-04-08	existing_value
-280	val_280	2008-04-08	existing_value
-2	val_2	2008-04-08	existing_value
-35	val_35	2008-04-08	existing_value
-283	val_283	2008-04-08	existing_value
-331	val_331	2008-04-08	existing_value
-235	val_235	2008-04-08	existing_value
-80	val_80	2008-04-08	existing_value
-44	val_44	2008-04-08	existing_value
-193	val_193	2008-04-08	existing_value
-321	val_321	2008-04-08	existing_value
-335	val_335	2008-04-08	existing_value
-104	val_104	2008-04-08	existing_value
-466	val_466	2008-04-08	existing_value
-366	val_366	2008-04-08	existing_value
-175	val_175	2008-04-08	existing_value
-403	val_403	2008-04-08	existing_value
-483	val_483	2008-04-08	existing_value
-53	val_53	2008-04-08	existing_value
-105	val_105	2008-04-08	existing_value
-257	val_257	2008-04-08	existing_value
-406	val_406	2008-04-08	existing_value
-409	val_409	2008-04-08	existing_value
-190	val_190	2008-04-08	existing_value
-406	val_406	2008-04-08	existing_value
-401	val_401	2008-04-08	existing_value
-114	val_114	2008-04-08	existing_value
-258	val_258	2008-04-08	existing_value
-90	val_90	2008-04-08	existing_value
-203	val_203	2008-04-08	existing_value
-262	val_262	2008-04-08	existing_value
-348	val_348	2008-04-08	existing_value
-424	val_424	2008-04-08	existing_value
-12	val_12	2008-04-08	existing_value
-396	val_396	2008-04-08	existing_value
-201	val_201	2008-04-08	existing_value
-217	val_217	2008-04-08	existing_value
-164	val_164	2008-04-08	existing_value
-431	val_431	2008-04-08	existing_value
-454	val_454	2008-04-08	existing_value
-478	val_478	2008-04-08	existing_value
-298	val_298	2008-04-08	existing_value
-125	val_125	2008-04-08	existing_value
-431	val_431	2008-04-08	existing_value
-164	val_164	2008-04-08	existing_value
-424	val_424	2008-04-08	existing_value
-187	val_187	2008-04-08	existing_value
-382	val_382	2008-04-08	existing_value
-5	val_5	2008-04-08	existing_value
-70	val_70	2008-04-08	existing_value
-397	val_397	2008-04-08	existing_value
-480	val_480	2008-04-08	existing_value
-291	val_291	2008-04-08	existing_value
-24	val_24	2008-04-08	existing_value
-351	val_351	2008-04-08	existing_value
-255	val_255	2008-04-08	existing_value
-104	val_104	2008-04-08	existing_value
-70	val_70	2008-04-08	existing_value
-163	val_163	2008-04-08	existing_value
-438	val_438	2008-04-08	existing_value
-119	val_119	2008-04-08	existing_value
-414	val_414	2008-04-08	existing_value
-200	val_200	2008-04-08	existing_value
-491	val_491	2008-04-08	existing_value
-237	val_237	2008-04-08	existing_value
-439	val_439	2008-04-08	existing_value
-360	val_360	2008-04-08	existing_value
-248	val_248	2008-04-08	existing_value
-479	val_479	2008-04-08	existing_value
-305	val_305	2008-04-08	existing_value
-417	val_417	2008-04-08	existing_value
-199	val_199	2008-04-08	existing_value
-444	val_444	2008-04-08	existing_value
-120	val_120	2008-04-08	existing_value
-429	val_429	2008-04-08	existing_value
-169	val_169	2008-04-08	existing_value
-443	val_443	2008-04-08	existing_value
-323	val_323	2008-04-08	existing_value
-325	val_325	2008-04-08	existing_value
-277	val_277	2008-04-08	existing_value
-230	val_230	2008-04-08	existing_value
-478	val_478	2008-04-08	existing_value
-178	val_178	2008-04-08	existing_value
-468	val_468	2008-04-08	existing_value
-310	val_310	2008-04-08	existing_value
-317	val_317	2008-04-08	existing_value
-333	val_333	2008-04-08	existing_value
-493	val_493	2008-04-08	existing_value
-460	val_460	2008-04-08	existing_value
-207	val_207	2008-04-08	existing_value
-249	val_249	2008-04-08	existing_value
-265	val_265	2008-04-08	existing_value
-480	val_480	2008-04-08	existing_value
-83	val_83	2008-04-08	existing_value
-136	val_136	2008-04-08	existing_value
-353	val_353	2008-04-08	existing_value
-172	val_172	2008-04-08	existing_value
-214	val_214	2008-04-08	existing_value
-462	val_462	2008-04-08	existing_value
-233	val_233	2008-04-08	existing_value
-406	val_406	2008-04-08	existing_value
-133	val_133	2008-04-08	existing_value
-175	val_175	2008-04-08	existing_value
-189	val_189	2008-04-08	existing_value
-454	val_454	2008-04-08	existing_value
-375	val_375	2008-04-08	existing_value
-401	val_401	2008-04-08	existing_value
-421	val_421	2008-04-08	existing_value
-407	val_407	2008-04-08	existing_value
-384	val_384	2008-04-08	existing_value
-256	val_256	2008-04-08	existing_value
-26	val_26	2008-04-08	existing_value
-134	val_134	2008-04-08	existing_value
-67	val_67	2008-04-08	existing_value
-384	val_384	2008-04-08	existing_value
-379	val_379	2008-04-08	existing_value
-18	val_18	2008-04-08	existing_value
-462	val_462	2008-04-08	existing_value
-492	val_492	2008-04-08	existing_value
-100	val_100	2008-04-08	existing_value
-298	val_298	2008-04-08	existing_value
-9	val_9	2008-04-08	existing_value
-341	val_341	2008-04-08	existing_value
-498	val_498	2008-04-08	existing_value
-146	val_146	2008-04-08	existing_value
-458	val_458	2008-04-08	existing_value
-362	val_362	2008-04-08	existing_value
-186	val_186	2008-04-08	existing_value
-285	val_285	2008-04-08	existing_value
-348	val_348	2008-04-08	existing_value
-167	val_167	2008-04-08	existing_value
-18	val_18	2008-04-08	existing_value
-273	val_273	2008-04-08	existing_value
-183	val_183	2008-04-08	existing_value
-281	val_281	2008-04-08	existing_value
-344	val_344	2008-04-08	existing_value
-97	val_97	2008-04-08	existing_value
-469	val_469	2008-04-08	existing_value
-315	val_315	2008-04-08	existing_value
-84	val_84	2008-04-08	existing_value
-28	val_28	2008-04-08	existing_value
-37	val_37	2008-04-08	existing_value
-448	val_448	2008-04-08	existing_value
-152	val_152	2008-04-08	existing_value
-348	val_348	2008-04-08	existing_value
-307	val_307	2008-04-08	existing_value
-194	val_194	2008-04-08	existing_value
-414	val_414	2008-04-08	existing_value
-477	val_477	2008-04-08	existing_value
-222	val_222	2008-04-08	existing_value
-126	val_126	2008-04-08	existing_value
-90	val_90	2008-04-08	existing_value
-169	val_169	2008-04-08	existing_value
-403	val_403	2008-04-08	existing_value
-400	val_400	2008-04-08	existing_value
-200	val_200	2008-04-08	existing_value
-97	val_97	2008-04-08	existing_value
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 b/sql/hive/src/test/resources/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7
deleted file mode 100644
index f1801743dd4e1f8d331aec922d57da77598b8521..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7
+++ /dev/null
@@ -1,2500 +0,0 @@
-238	val_238	2008-04-08	11
-86	val_86	2008-04-08	11
-311	val_311	2008-04-08	11
-27	val_27	2008-04-08	11
-165	val_165	2008-04-08	11
-409	val_409	2008-04-08	11
-255	val_255	2008-04-08	11
-278	val_278	2008-04-08	11
-98	val_98	2008-04-08	11
-484	val_484	2008-04-08	11
-265	val_265	2008-04-08	11
-193	val_193	2008-04-08	11
-401	val_401	2008-04-08	11
-150	val_150	2008-04-08	11
-273	val_273	2008-04-08	11
-224	val_224	2008-04-08	11
-369	val_369	2008-04-08	11
-66	val_66	2008-04-08	11
-128	val_128	2008-04-08	11
-213	val_213	2008-04-08	11
-146	val_146	2008-04-08	11
-406	val_406	2008-04-08	11
-429	val_429	2008-04-08	11
-374	val_374	2008-04-08	11
-152	val_152	2008-04-08	11
-469	val_469	2008-04-08	11
-145	val_145	2008-04-08	11
-495	val_495	2008-04-08	11
-37	val_37	2008-04-08	11
-327	val_327	2008-04-08	11
-281	val_281	2008-04-08	11
-277	val_277	2008-04-08	11
-209	val_209	2008-04-08	11
-15	val_15	2008-04-08	11
-82	val_82	2008-04-08	11
-403	val_403	2008-04-08	11
-166	val_166	2008-04-08	11
-417	val_417	2008-04-08	11
-430	val_430	2008-04-08	11
-252	val_252	2008-04-08	11
-292	val_292	2008-04-08	11
-219	val_219	2008-04-08	11
-287	val_287	2008-04-08	11
-153	val_153	2008-04-08	11
-193	val_193	2008-04-08	11
-338	val_338	2008-04-08	11
-446	val_446	2008-04-08	11
-459	val_459	2008-04-08	11
-394	val_394	2008-04-08	11
-237	val_237	2008-04-08	11
-482	val_482	2008-04-08	11
-174	val_174	2008-04-08	11
-413	val_413	2008-04-08	11
-494	val_494	2008-04-08	11
-207	val_207	2008-04-08	11
-199	val_199	2008-04-08	11
-466	val_466	2008-04-08	11
-208	val_208	2008-04-08	11
-174	val_174	2008-04-08	11
-399	val_399	2008-04-08	11
-396	val_396	2008-04-08	11
-247	val_247	2008-04-08	11
-417	val_417	2008-04-08	11
-489	val_489	2008-04-08	11
-162	val_162	2008-04-08	11
-377	val_377	2008-04-08	11
-397	val_397	2008-04-08	11
-309	val_309	2008-04-08	11
-365	val_365	2008-04-08	11
-266	val_266	2008-04-08	11
-439	val_439	2008-04-08	11
-342	val_342	2008-04-08	11
-367	val_367	2008-04-08	11
-325	val_325	2008-04-08	11
-167	val_167	2008-04-08	11
-195	val_195	2008-04-08	11
-475	val_475	2008-04-08	11
-17	val_17	2008-04-08	11
-113	val_113	2008-04-08	11
-155	val_155	2008-04-08	11
-203	val_203	2008-04-08	11
-339	val_339	2008-04-08	11
-0	val_0	2008-04-08	11
-455	val_455	2008-04-08	11
-128	val_128	2008-04-08	11
-311	val_311	2008-04-08	11
-316	val_316	2008-04-08	11
-57	val_57	2008-04-08	11
-302	val_302	2008-04-08	11
-205	val_205	2008-04-08	11
-149	val_149	2008-04-08	11
-438	val_438	2008-04-08	11
-345	val_345	2008-04-08	11
-129	val_129	2008-04-08	11
-170	val_170	2008-04-08	11
-20	val_20	2008-04-08	11
-489	val_489	2008-04-08	11
-157	val_157	2008-04-08	11
-378	val_378	2008-04-08	11
-221	val_221	2008-04-08	11
-92	val_92	2008-04-08	11
-111	val_111	2008-04-08	11
-47	val_47	2008-04-08	11
-72	val_72	2008-04-08	11
-4	val_4	2008-04-08	11
-280	val_280	2008-04-08	11
-35	val_35	2008-04-08	11
-427	val_427	2008-04-08	11
-277	val_277	2008-04-08	11
-208	val_208	2008-04-08	11
-356	val_356	2008-04-08	11
-399	val_399	2008-04-08	11
-169	val_169	2008-04-08	11
-382	val_382	2008-04-08	11
-498	val_498	2008-04-08	11
-125	val_125	2008-04-08	11
-386	val_386	2008-04-08	11
-437	val_437	2008-04-08	11
-469	val_469	2008-04-08	11
-192	val_192	2008-04-08	11
-286	val_286	2008-04-08	11
-187	val_187	2008-04-08	11
-176	val_176	2008-04-08	11
-54	val_54	2008-04-08	11
-459	val_459	2008-04-08	11
-51	val_51	2008-04-08	11
-138	val_138	2008-04-08	11
-103	val_103	2008-04-08	11
-239	val_239	2008-04-08	11
-213	val_213	2008-04-08	11
-216	val_216	2008-04-08	11
-430	val_430	2008-04-08	11
-278	val_278	2008-04-08	11
-176	val_176	2008-04-08	11
-289	val_289	2008-04-08	11
-221	val_221	2008-04-08	11
-65	val_65	2008-04-08	11
-318	val_318	2008-04-08	11
-332	val_332	2008-04-08	11
-311	val_311	2008-04-08	11
-275	val_275	2008-04-08	11
-137	val_137	2008-04-08	11
-241	val_241	2008-04-08	11
-83	val_83	2008-04-08	11
-333	val_333	2008-04-08	11
-180	val_180	2008-04-08	11
-284	val_284	2008-04-08	11
-12	val_12	2008-04-08	11
-230	val_230	2008-04-08	11
-181	val_181	2008-04-08	11
-67	val_67	2008-04-08	11
-260	val_260	2008-04-08	11
-404	val_404	2008-04-08	11
-384	val_384	2008-04-08	11
-489	val_489	2008-04-08	11
-353	val_353	2008-04-08	11
-373	val_373	2008-04-08	11
-272	val_272	2008-04-08	11
-138	val_138	2008-04-08	11
-217	val_217	2008-04-08	11
-84	val_84	2008-04-08	11
-348	val_348	2008-04-08	11
-466	val_466	2008-04-08	11
-58	val_58	2008-04-08	11
-8	val_8	2008-04-08	11
-411	val_411	2008-04-08	11
-230	val_230	2008-04-08	11
-208	val_208	2008-04-08	11
-348	val_348	2008-04-08	11
-24	val_24	2008-04-08	11
-463	val_463	2008-04-08	11
-431	val_431	2008-04-08	11
-179	val_179	2008-04-08	11
-172	val_172	2008-04-08	11
-42	val_42	2008-04-08	11
-129	val_129	2008-04-08	11
-158	val_158	2008-04-08	11
-119	val_119	2008-04-08	11
-496	val_496	2008-04-08	11
-0	val_0	2008-04-08	11
-322	val_322	2008-04-08	11
-197	val_197	2008-04-08	11
-468	val_468	2008-04-08	11
-393	val_393	2008-04-08	11
-454	val_454	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-199	val_199	2008-04-08	11
-191	val_191	2008-04-08	11
-418	val_418	2008-04-08	11
-96	val_96	2008-04-08	11
-26	val_26	2008-04-08	11
-165	val_165	2008-04-08	11
-327	val_327	2008-04-08	11
-230	val_230	2008-04-08	11
-205	val_205	2008-04-08	11
-120	val_120	2008-04-08	11
-131	val_131	2008-04-08	11
-51	val_51	2008-04-08	11
-404	val_404	2008-04-08	11
-43	val_43	2008-04-08	11
-436	val_436	2008-04-08	11
-156	val_156	2008-04-08	11
-469	val_469	2008-04-08	11
-468	val_468	2008-04-08	11
-308	val_308	2008-04-08	11
-95	val_95	2008-04-08	11
-196	val_196	2008-04-08	11
-288	val_288	2008-04-08	11
-481	val_481	2008-04-08	11
-457	val_457	2008-04-08	11
-98	val_98	2008-04-08	11
-282	val_282	2008-04-08	11
-197	val_197	2008-04-08	11
-187	val_187	2008-04-08	11
-318	val_318	2008-04-08	11
-318	val_318	2008-04-08	11
-409	val_409	2008-04-08	11
-470	val_470	2008-04-08	11
-137	val_137	2008-04-08	11
-369	val_369	2008-04-08	11
-316	val_316	2008-04-08	11
-169	val_169	2008-04-08	11
-413	val_413	2008-04-08	11
-85	val_85	2008-04-08	11
-77	val_77	2008-04-08	11
-0	val_0	2008-04-08	11
-490	val_490	2008-04-08	11
-87	val_87	2008-04-08	11
-364	val_364	2008-04-08	11
-179	val_179	2008-04-08	11
-118	val_118	2008-04-08	11
-134	val_134	2008-04-08	11
-395	val_395	2008-04-08	11
-282	val_282	2008-04-08	11
-138	val_138	2008-04-08	11
-238	val_238	2008-04-08	11
-419	val_419	2008-04-08	11
-15	val_15	2008-04-08	11
-118	val_118	2008-04-08	11
-72	val_72	2008-04-08	11
-90	val_90	2008-04-08	11
-307	val_307	2008-04-08	11
-19	val_19	2008-04-08	11
-435	val_435	2008-04-08	11
-10	val_10	2008-04-08	11
-277	val_277	2008-04-08	11
-273	val_273	2008-04-08	11
-306	val_306	2008-04-08	11
-224	val_224	2008-04-08	11
-309	val_309	2008-04-08	11
-389	val_389	2008-04-08	11
-327	val_327	2008-04-08	11
-242	val_242	2008-04-08	11
-369	val_369	2008-04-08	11
-392	val_392	2008-04-08	11
-272	val_272	2008-04-08	11
-331	val_331	2008-04-08	11
-401	val_401	2008-04-08	11
-242	val_242	2008-04-08	11
-452	val_452	2008-04-08	11
-177	val_177	2008-04-08	11
-226	val_226	2008-04-08	11
-5	val_5	2008-04-08	11
-497	val_497	2008-04-08	11
-402	val_402	2008-04-08	11
-396	val_396	2008-04-08	11
-317	val_317	2008-04-08	11
-395	val_395	2008-04-08	11
-58	val_58	2008-04-08	11
-35	val_35	2008-04-08	11
-336	val_336	2008-04-08	11
-95	val_95	2008-04-08	11
-11	val_11	2008-04-08	11
-168	val_168	2008-04-08	11
-34	val_34	2008-04-08	11
-229	val_229	2008-04-08	11
-233	val_233	2008-04-08	11
-143	val_143	2008-04-08	11
-472	val_472	2008-04-08	11
-322	val_322	2008-04-08	11
-498	val_498	2008-04-08	11
-160	val_160	2008-04-08	11
-195	val_195	2008-04-08	11
-42	val_42	2008-04-08	11
-321	val_321	2008-04-08	11
-430	val_430	2008-04-08	11
-119	val_119	2008-04-08	11
-489	val_489	2008-04-08	11
-458	val_458	2008-04-08	11
-78	val_78	2008-04-08	11
-76	val_76	2008-04-08	11
-41	val_41	2008-04-08	11
-223	val_223	2008-04-08	11
-492	val_492	2008-04-08	11
-149	val_149	2008-04-08	11
-449	val_449	2008-04-08	11
-218	val_218	2008-04-08	11
-228	val_228	2008-04-08	11
-138	val_138	2008-04-08	11
-453	val_453	2008-04-08	11
-30	val_30	2008-04-08	11
-209	val_209	2008-04-08	11
-64	val_64	2008-04-08	11
-468	val_468	2008-04-08	11
-76	val_76	2008-04-08	11
-74	val_74	2008-04-08	11
-342	val_342	2008-04-08	11
-69	val_69	2008-04-08	11
-230	val_230	2008-04-08	11
-33	val_33	2008-04-08	11
-368	val_368	2008-04-08	11
-103	val_103	2008-04-08	11
-296	val_296	2008-04-08	11
-113	val_113	2008-04-08	11
-216	val_216	2008-04-08	11
-367	val_367	2008-04-08	11
-344	val_344	2008-04-08	11
-167	val_167	2008-04-08	11
-274	val_274	2008-04-08	11
-219	val_219	2008-04-08	11
-239	val_239	2008-04-08	11
-485	val_485	2008-04-08	11
-116	val_116	2008-04-08	11
-223	val_223	2008-04-08	11
-256	val_256	2008-04-08	11
-263	val_263	2008-04-08	11
-70	val_70	2008-04-08	11
-487	val_487	2008-04-08	11
-480	val_480	2008-04-08	11
-401	val_401	2008-04-08	11
-288	val_288	2008-04-08	11
-191	val_191	2008-04-08	11
-5	val_5	2008-04-08	11
-244	val_244	2008-04-08	11
-438	val_438	2008-04-08	11
-128	val_128	2008-04-08	11
-467	val_467	2008-04-08	11
-432	val_432	2008-04-08	11
-202	val_202	2008-04-08	11
-316	val_316	2008-04-08	11
-229	val_229	2008-04-08	11
-469	val_469	2008-04-08	11
-463	val_463	2008-04-08	11
-280	val_280	2008-04-08	11
-2	val_2	2008-04-08	11
-35	val_35	2008-04-08	11
-283	val_283	2008-04-08	11
-331	val_331	2008-04-08	11
-235	val_235	2008-04-08	11
-80	val_80	2008-04-08	11
-44	val_44	2008-04-08	11
-193	val_193	2008-04-08	11
-321	val_321	2008-04-08	11
-335	val_335	2008-04-08	11
-104	val_104	2008-04-08	11
-466	val_466	2008-04-08	11
-366	val_366	2008-04-08	11
-175	val_175	2008-04-08	11
-403	val_403	2008-04-08	11
-483	val_483	2008-04-08	11
-53	val_53	2008-04-08	11
-105	val_105	2008-04-08	11
-257	val_257	2008-04-08	11
-406	val_406	2008-04-08	11
-409	val_409	2008-04-08	11
-190	val_190	2008-04-08	11
-406	val_406	2008-04-08	11
-401	val_401	2008-04-08	11
-114	val_114	2008-04-08	11
-258	val_258	2008-04-08	11
-90	val_90	2008-04-08	11
-203	val_203	2008-04-08	11
-262	val_262	2008-04-08	11
-348	val_348	2008-04-08	11
-424	val_424	2008-04-08	11
-12	val_12	2008-04-08	11
-396	val_396	2008-04-08	11
-201	val_201	2008-04-08	11
-217	val_217	2008-04-08	11
-164	val_164	2008-04-08	11
-431	val_431	2008-04-08	11
-454	val_454	2008-04-08	11
-478	val_478	2008-04-08	11
-298	val_298	2008-04-08	11
-125	val_125	2008-04-08	11
-431	val_431	2008-04-08	11
-164	val_164	2008-04-08	11
-424	val_424	2008-04-08	11
-187	val_187	2008-04-08	11
-382	val_382	2008-04-08	11
-5	val_5	2008-04-08	11
-70	val_70	2008-04-08	11
-397	val_397	2008-04-08	11
-480	val_480	2008-04-08	11
-291	val_291	2008-04-08	11
-24	val_24	2008-04-08	11
-351	val_351	2008-04-08	11
-255	val_255	2008-04-08	11
-104	val_104	2008-04-08	11
-70	val_70	2008-04-08	11
-163	val_163	2008-04-08	11
-438	val_438	2008-04-08	11
-119	val_119	2008-04-08	11
-414	val_414	2008-04-08	11
-200	val_200	2008-04-08	11
-491	val_491	2008-04-08	11
-237	val_237	2008-04-08	11
-439	val_439	2008-04-08	11
-360	val_360	2008-04-08	11
-248	val_248	2008-04-08	11
-479	val_479	2008-04-08	11
-305	val_305	2008-04-08	11
-417	val_417	2008-04-08	11
-199	val_199	2008-04-08	11
-444	val_444	2008-04-08	11
-120	val_120	2008-04-08	11
-429	val_429	2008-04-08	11
-169	val_169	2008-04-08	11
-443	val_443	2008-04-08	11
-323	val_323	2008-04-08	11
-325	val_325	2008-04-08	11
-277	val_277	2008-04-08	11
-230	val_230	2008-04-08	11
-478	val_478	2008-04-08	11
-178	val_178	2008-04-08	11
-468	val_468	2008-04-08	11
-310	val_310	2008-04-08	11
-317	val_317	2008-04-08	11
-333	val_333	2008-04-08	11
-493	val_493	2008-04-08	11
-460	val_460	2008-04-08	11
-207	val_207	2008-04-08	11
-249	val_249	2008-04-08	11
-265	val_265	2008-04-08	11
-480	val_480	2008-04-08	11
-83	val_83	2008-04-08	11
-136	val_136	2008-04-08	11
-353	val_353	2008-04-08	11
-172	val_172	2008-04-08	11
-214	val_214	2008-04-08	11
-462	val_462	2008-04-08	11
-233	val_233	2008-04-08	11
-406	val_406	2008-04-08	11
-133	val_133	2008-04-08	11
-175	val_175	2008-04-08	11
-189	val_189	2008-04-08	11
-454	val_454	2008-04-08	11
-375	val_375	2008-04-08	11
-401	val_401	2008-04-08	11
-421	val_421	2008-04-08	11
-407	val_407	2008-04-08	11
-384	val_384	2008-04-08	11
-256	val_256	2008-04-08	11
-26	val_26	2008-04-08	11
-134	val_134	2008-04-08	11
-67	val_67	2008-04-08	11
-384	val_384	2008-04-08	11
-379	val_379	2008-04-08	11
-18	val_18	2008-04-08	11
-462	val_462	2008-04-08	11
-492	val_492	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-9	val_9	2008-04-08	11
-341	val_341	2008-04-08	11
-498	val_498	2008-04-08	11
-146	val_146	2008-04-08	11
-458	val_458	2008-04-08	11
-362	val_362	2008-04-08	11
-186	val_186	2008-04-08	11
-285	val_285	2008-04-08	11
-348	val_348	2008-04-08	11
-167	val_167	2008-04-08	11
-18	val_18	2008-04-08	11
-273	val_273	2008-04-08	11
-183	val_183	2008-04-08	11
-281	val_281	2008-04-08	11
-344	val_344	2008-04-08	11
-97	val_97	2008-04-08	11
-469	val_469	2008-04-08	11
-315	val_315	2008-04-08	11
-84	val_84	2008-04-08	11
-28	val_28	2008-04-08	11
-37	val_37	2008-04-08	11
-448	val_448	2008-04-08	11
-152	val_152	2008-04-08	11
-348	val_348	2008-04-08	11
-307	val_307	2008-04-08	11
-194	val_194	2008-04-08	11
-414	val_414	2008-04-08	11
-477	val_477	2008-04-08	11
-222	val_222	2008-04-08	11
-126	val_126	2008-04-08	11
-90	val_90	2008-04-08	11
-169	val_169	2008-04-08	11
-403	val_403	2008-04-08	11
-400	val_400	2008-04-08	11
-200	val_200	2008-04-08	11
-97	val_97	2008-04-08	11
-238	val_238	2008-04-08	12
-86	val_86	2008-04-08	12
-311	val_311	2008-04-08	12
-27	val_27	2008-04-08	12
-165	val_165	2008-04-08	12
-409	val_409	2008-04-08	12
-255	val_255	2008-04-08	12
-278	val_278	2008-04-08	12
-98	val_98	2008-04-08	12
-484	val_484	2008-04-08	12
-265	val_265	2008-04-08	12
-193	val_193	2008-04-08	12
-401	val_401	2008-04-08	12
-150	val_150	2008-04-08	12
-273	val_273	2008-04-08	12
-224	val_224	2008-04-08	12
-369	val_369	2008-04-08	12
-66	val_66	2008-04-08	12
-128	val_128	2008-04-08	12
-213	val_213	2008-04-08	12
-146	val_146	2008-04-08	12
-406	val_406	2008-04-08	12
-429	val_429	2008-04-08	12
-374	val_374	2008-04-08	12
-152	val_152	2008-04-08	12
-469	val_469	2008-04-08	12
-145	val_145	2008-04-08	12
-495	val_495	2008-04-08	12
-37	val_37	2008-04-08	12
-327	val_327	2008-04-08	12
-281	val_281	2008-04-08	12
-277	val_277	2008-04-08	12
-209	val_209	2008-04-08	12
-15	val_15	2008-04-08	12
-82	val_82	2008-04-08	12
-403	val_403	2008-04-08	12
-166	val_166	2008-04-08	12
-417	val_417	2008-04-08	12
-430	val_430	2008-04-08	12
-252	val_252	2008-04-08	12
-292	val_292	2008-04-08	12
-219	val_219	2008-04-08	12
-287	val_287	2008-04-08	12
-153	val_153	2008-04-08	12
-193	val_193	2008-04-08	12
-338	val_338	2008-04-08	12
-446	val_446	2008-04-08	12
-459	val_459	2008-04-08	12
-394	val_394	2008-04-08	12
-237	val_237	2008-04-08	12
-482	val_482	2008-04-08	12
-174	val_174	2008-04-08	12
-413	val_413	2008-04-08	12
-494	val_494	2008-04-08	12
-207	val_207	2008-04-08	12
-199	val_199	2008-04-08	12
-466	val_466	2008-04-08	12
-208	val_208	2008-04-08	12
-174	val_174	2008-04-08	12
-399	val_399	2008-04-08	12
-396	val_396	2008-04-08	12
-247	val_247	2008-04-08	12
-417	val_417	2008-04-08	12
-489	val_489	2008-04-08	12
-162	val_162	2008-04-08	12
-377	val_377	2008-04-08	12
-397	val_397	2008-04-08	12
-309	val_309	2008-04-08	12
-365	val_365	2008-04-08	12
-266	val_266	2008-04-08	12
-439	val_439	2008-04-08	12
-342	val_342	2008-04-08	12
-367	val_367	2008-04-08	12
-325	val_325	2008-04-08	12
-167	val_167	2008-04-08	12
-195	val_195	2008-04-08	12
-475	val_475	2008-04-08	12
-17	val_17	2008-04-08	12
-113	val_113	2008-04-08	12
-155	val_155	2008-04-08	12
-203	val_203	2008-04-08	12
-339	val_339	2008-04-08	12
-0	val_0	2008-04-08	12
-455	val_455	2008-04-08	12
-128	val_128	2008-04-08	12
-311	val_311	2008-04-08	12
-316	val_316	2008-04-08	12
-57	val_57	2008-04-08	12
-302	val_302	2008-04-08	12
-205	val_205	2008-04-08	12
-149	val_149	2008-04-08	12
-438	val_438	2008-04-08	12
-345	val_345	2008-04-08	12
-129	val_129	2008-04-08	12
-170	val_170	2008-04-08	12
-20	val_20	2008-04-08	12
-489	val_489	2008-04-08	12
-157	val_157	2008-04-08	12
-378	val_378	2008-04-08	12
-221	val_221	2008-04-08	12
-92	val_92	2008-04-08	12
-111	val_111	2008-04-08	12
-47	val_47	2008-04-08	12
-72	val_72	2008-04-08	12
-4	val_4	2008-04-08	12
-280	val_280	2008-04-08	12
-35	val_35	2008-04-08	12
-427	val_427	2008-04-08	12
-277	val_277	2008-04-08	12
-208	val_208	2008-04-08	12
-356	val_356	2008-04-08	12
-399	val_399	2008-04-08	12
-169	val_169	2008-04-08	12
-382	val_382	2008-04-08	12
-498	val_498	2008-04-08	12
-125	val_125	2008-04-08	12
-386	val_386	2008-04-08	12
-437	val_437	2008-04-08	12
-469	val_469	2008-04-08	12
-192	val_192	2008-04-08	12
-286	val_286	2008-04-08	12
-187	val_187	2008-04-08	12
-176	val_176	2008-04-08	12
-54	val_54	2008-04-08	12
-459	val_459	2008-04-08	12
-51	val_51	2008-04-08	12
-138	val_138	2008-04-08	12
-103	val_103	2008-04-08	12
-239	val_239	2008-04-08	12
-213	val_213	2008-04-08	12
-216	val_216	2008-04-08	12
-430	val_430	2008-04-08	12
-278	val_278	2008-04-08	12
-176	val_176	2008-04-08	12
-289	val_289	2008-04-08	12
-221	val_221	2008-04-08	12
-65	val_65	2008-04-08	12
-318	val_318	2008-04-08	12
-332	val_332	2008-04-08	12
-311	val_311	2008-04-08	12
-275	val_275	2008-04-08	12
-137	val_137	2008-04-08	12
-241	val_241	2008-04-08	12
-83	val_83	2008-04-08	12
-333	val_333	2008-04-08	12
-180	val_180	2008-04-08	12
-284	val_284	2008-04-08	12
-12	val_12	2008-04-08	12
-230	val_230	2008-04-08	12
-181	val_181	2008-04-08	12
-67	val_67	2008-04-08	12
-260	val_260	2008-04-08	12
-404	val_404	2008-04-08	12
-384	val_384	2008-04-08	12
-489	val_489	2008-04-08	12
-353	val_353	2008-04-08	12
-373	val_373	2008-04-08	12
-272	val_272	2008-04-08	12
-138	val_138	2008-04-08	12
-217	val_217	2008-04-08	12
-84	val_84	2008-04-08	12
-348	val_348	2008-04-08	12
-466	val_466	2008-04-08	12
-58	val_58	2008-04-08	12
-8	val_8	2008-04-08	12
-411	val_411	2008-04-08	12
-230	val_230	2008-04-08	12
-208	val_208	2008-04-08	12
-348	val_348	2008-04-08	12
-24	val_24	2008-04-08	12
-463	val_463	2008-04-08	12
-431	val_431	2008-04-08	12
-179	val_179	2008-04-08	12
-172	val_172	2008-04-08	12
-42	val_42	2008-04-08	12
-129	val_129	2008-04-08	12
-158	val_158	2008-04-08	12
-119	val_119	2008-04-08	12
-496	val_496	2008-04-08	12
-0	val_0	2008-04-08	12
-322	val_322	2008-04-08	12
-197	val_197	2008-04-08	12
-468	val_468	2008-04-08	12
-393	val_393	2008-04-08	12
-454	val_454	2008-04-08	12
-100	val_100	2008-04-08	12
-298	val_298	2008-04-08	12
-199	val_199	2008-04-08	12
-191	val_191	2008-04-08	12
-418	val_418	2008-04-08	12
-96	val_96	2008-04-08	12
-26	val_26	2008-04-08	12
-165	val_165	2008-04-08	12
-327	val_327	2008-04-08	12
-230	val_230	2008-04-08	12
-205	val_205	2008-04-08	12
-120	val_120	2008-04-08	12
-131	val_131	2008-04-08	12
-51	val_51	2008-04-08	12
-404	val_404	2008-04-08	12
-43	val_43	2008-04-08	12
-436	val_436	2008-04-08	12
-156	val_156	2008-04-08	12
-469	val_469	2008-04-08	12
-468	val_468	2008-04-08	12
-308	val_308	2008-04-08	12
-95	val_95	2008-04-08	12
-196	val_196	2008-04-08	12
-288	val_288	2008-04-08	12
-481	val_481	2008-04-08	12
-457	val_457	2008-04-08	12
-98	val_98	2008-04-08	12
-282	val_282	2008-04-08	12
-197	val_197	2008-04-08	12
-187	val_187	2008-04-08	12
-318	val_318	2008-04-08	12
-318	val_318	2008-04-08	12
-409	val_409	2008-04-08	12
-470	val_470	2008-04-08	12
-137	val_137	2008-04-08	12
-369	val_369	2008-04-08	12
-316	val_316	2008-04-08	12
-169	val_169	2008-04-08	12
-413	val_413	2008-04-08	12
-85	val_85	2008-04-08	12
-77	val_77	2008-04-08	12
-0	val_0	2008-04-08	12
-490	val_490	2008-04-08	12
-87	val_87	2008-04-08	12
-364	val_364	2008-04-08	12
-179	val_179	2008-04-08	12
-118	val_118	2008-04-08	12
-134	val_134	2008-04-08	12
-395	val_395	2008-04-08	12
-282	val_282	2008-04-08	12
-138	val_138	2008-04-08	12
-238	val_238	2008-04-08	12
-419	val_419	2008-04-08	12
-15	val_15	2008-04-08	12
-118	val_118	2008-04-08	12
-72	val_72	2008-04-08	12
-90	val_90	2008-04-08	12
-307	val_307	2008-04-08	12
-19	val_19	2008-04-08	12
-435	val_435	2008-04-08	12
-10	val_10	2008-04-08	12
-277	val_277	2008-04-08	12
-273	val_273	2008-04-08	12
-306	val_306	2008-04-08	12
-224	val_224	2008-04-08	12
-309	val_309	2008-04-08	12
-389	val_389	2008-04-08	12
-327	val_327	2008-04-08	12
-242	val_242	2008-04-08	12
-369	val_369	2008-04-08	12
-392	val_392	2008-04-08	12
-272	val_272	2008-04-08	12
-331	val_331	2008-04-08	12
-401	val_401	2008-04-08	12
-242	val_242	2008-04-08	12
-452	val_452	2008-04-08	12
-177	val_177	2008-04-08	12
-226	val_226	2008-04-08	12
-5	val_5	2008-04-08	12
-497	val_497	2008-04-08	12
-402	val_402	2008-04-08	12
-396	val_396	2008-04-08	12
-317	val_317	2008-04-08	12
-395	val_395	2008-04-08	12
-58	val_58	2008-04-08	12
-35	val_35	2008-04-08	12
-336	val_336	2008-04-08	12
-95	val_95	2008-04-08	12
-11	val_11	2008-04-08	12
-168	val_168	2008-04-08	12
-34	val_34	2008-04-08	12
-229	val_229	2008-04-08	12
-233	val_233	2008-04-08	12
-143	val_143	2008-04-08	12
-472	val_472	2008-04-08	12
-322	val_322	2008-04-08	12
-498	val_498	2008-04-08	12
-160	val_160	2008-04-08	12
-195	val_195	2008-04-08	12
-42	val_42	2008-04-08	12
-321	val_321	2008-04-08	12
-430	val_430	2008-04-08	12
-119	val_119	2008-04-08	12
-489	val_489	2008-04-08	12
-458	val_458	2008-04-08	12
-78	val_78	2008-04-08	12
-76	val_76	2008-04-08	12
-41	val_41	2008-04-08	12
-223	val_223	2008-04-08	12
-492	val_492	2008-04-08	12
-149	val_149	2008-04-08	12
-449	val_449	2008-04-08	12
-218	val_218	2008-04-08	12
-228	val_228	2008-04-08	12
-138	val_138	2008-04-08	12
-453	val_453	2008-04-08	12
-30	val_30	2008-04-08	12
-209	val_209	2008-04-08	12
-64	val_64	2008-04-08	12
-468	val_468	2008-04-08	12
-76	val_76	2008-04-08	12
-74	val_74	2008-04-08	12
-342	val_342	2008-04-08	12
-69	val_69	2008-04-08	12
-230	val_230	2008-04-08	12
-33	val_33	2008-04-08	12
-368	val_368	2008-04-08	12
-103	val_103	2008-04-08	12
-296	val_296	2008-04-08	12
-113	val_113	2008-04-08	12
-216	val_216	2008-04-08	12
-367	val_367	2008-04-08	12
-344	val_344	2008-04-08	12
-167	val_167	2008-04-08	12
-274	val_274	2008-04-08	12
-219	val_219	2008-04-08	12
-239	val_239	2008-04-08	12
-485	val_485	2008-04-08	12
-116	val_116	2008-04-08	12
-223	val_223	2008-04-08	12
-256	val_256	2008-04-08	12
-263	val_263	2008-04-08	12
-70	val_70	2008-04-08	12
-487	val_487	2008-04-08	12
-480	val_480	2008-04-08	12
-401	val_401	2008-04-08	12
-288	val_288	2008-04-08	12
-191	val_191	2008-04-08	12
-5	val_5	2008-04-08	12
-244	val_244	2008-04-08	12
-438	val_438	2008-04-08	12
-128	val_128	2008-04-08	12
-467	val_467	2008-04-08	12
-432	val_432	2008-04-08	12
-202	val_202	2008-04-08	12
-316	val_316	2008-04-08	12
-229	val_229	2008-04-08	12
-469	val_469	2008-04-08	12
-463	val_463	2008-04-08	12
-280	val_280	2008-04-08	12
-2	val_2	2008-04-08	12
-35	val_35	2008-04-08	12
-283	val_283	2008-04-08	12
-331	val_331	2008-04-08	12
-235	val_235	2008-04-08	12
-80	val_80	2008-04-08	12
-44	val_44	2008-04-08	12
-193	val_193	2008-04-08	12
-321	val_321	2008-04-08	12
-335	val_335	2008-04-08	12
-104	val_104	2008-04-08	12
-466	val_466	2008-04-08	12
-366	val_366	2008-04-08	12
-175	val_175	2008-04-08	12
-403	val_403	2008-04-08	12
-483	val_483	2008-04-08	12
-53	val_53	2008-04-08	12
-105	val_105	2008-04-08	12
-257	val_257	2008-04-08	12
-406	val_406	2008-04-08	12
-409	val_409	2008-04-08	12
-190	val_190	2008-04-08	12
-406	val_406	2008-04-08	12
-401	val_401	2008-04-08	12
-114	val_114	2008-04-08	12
-258	val_258	2008-04-08	12
-90	val_90	2008-04-08	12
-203	val_203	2008-04-08	12
-262	val_262	2008-04-08	12
-348	val_348	2008-04-08	12
-424	val_424	2008-04-08	12
-12	val_12	2008-04-08	12
-396	val_396	2008-04-08	12
-201	val_201	2008-04-08	12
-217	val_217	2008-04-08	12
-164	val_164	2008-04-08	12
-431	val_431	2008-04-08	12
-454	val_454	2008-04-08	12
-478	val_478	2008-04-08	12
-298	val_298	2008-04-08	12
-125	val_125	2008-04-08	12
-431	val_431	2008-04-08	12
-164	val_164	2008-04-08	12
-424	val_424	2008-04-08	12
-187	val_187	2008-04-08	12
-382	val_382	2008-04-08	12
-5	val_5	2008-04-08	12
-70	val_70	2008-04-08	12
-397	val_397	2008-04-08	12
-480	val_480	2008-04-08	12
-291	val_291	2008-04-08	12
-24	val_24	2008-04-08	12
-351	val_351	2008-04-08	12
-255	val_255	2008-04-08	12
-104	val_104	2008-04-08	12
-70	val_70	2008-04-08	12
-163	val_163	2008-04-08	12
-438	val_438	2008-04-08	12
-119	val_119	2008-04-08	12
-414	val_414	2008-04-08	12
-200	val_200	2008-04-08	12
-491	val_491	2008-04-08	12
-237	val_237	2008-04-08	12
-439	val_439	2008-04-08	12
-360	val_360	2008-04-08	12
-248	val_248	2008-04-08	12
-479	val_479	2008-04-08	12
-305	val_305	2008-04-08	12
-417	val_417	2008-04-08	12
-199	val_199	2008-04-08	12
-444	val_444	2008-04-08	12
-120	val_120	2008-04-08	12
-429	val_429	2008-04-08	12
-169	val_169	2008-04-08	12
-443	val_443	2008-04-08	12
-323	val_323	2008-04-08	12
-325	val_325	2008-04-08	12
-277	val_277	2008-04-08	12
-230	val_230	2008-04-08	12
-478	val_478	2008-04-08	12
-178	val_178	2008-04-08	12
-468	val_468	2008-04-08	12
-310	val_310	2008-04-08	12
-317	val_317	2008-04-08	12
-333	val_333	2008-04-08	12
-493	val_493	2008-04-08	12
-460	val_460	2008-04-08	12
-207	val_207	2008-04-08	12
-249	val_249	2008-04-08	12
-265	val_265	2008-04-08	12
-480	val_480	2008-04-08	12
-83	val_83	2008-04-08	12
-136	val_136	2008-04-08	12
-353	val_353	2008-04-08	12
-172	val_172	2008-04-08	12
-214	val_214	2008-04-08	12
-462	val_462	2008-04-08	12
-233	val_233	2008-04-08	12
-406	val_406	2008-04-08	12
-133	val_133	2008-04-08	12
-175	val_175	2008-04-08	12
-189	val_189	2008-04-08	12
-454	val_454	2008-04-08	12
-375	val_375	2008-04-08	12
-401	val_401	2008-04-08	12
-421	val_421	2008-04-08	12
-407	val_407	2008-04-08	12
-384	val_384	2008-04-08	12
-256	val_256	2008-04-08	12
-26	val_26	2008-04-08	12
-134	val_134	2008-04-08	12
-67	val_67	2008-04-08	12
-384	val_384	2008-04-08	12
-379	val_379	2008-04-08	12
-18	val_18	2008-04-08	12
-462	val_462	2008-04-08	12
-492	val_492	2008-04-08	12
-100	val_100	2008-04-08	12
-298	val_298	2008-04-08	12
-9	val_9	2008-04-08	12
-341	val_341	2008-04-08	12
-498	val_498	2008-04-08	12
-146	val_146	2008-04-08	12
-458	val_458	2008-04-08	12
-362	val_362	2008-04-08	12
-186	val_186	2008-04-08	12
-285	val_285	2008-04-08	12
-348	val_348	2008-04-08	12
-167	val_167	2008-04-08	12
-18	val_18	2008-04-08	12
-273	val_273	2008-04-08	12
-183	val_183	2008-04-08	12
-281	val_281	2008-04-08	12
-344	val_344	2008-04-08	12
-97	val_97	2008-04-08	12
-469	val_469	2008-04-08	12
-315	val_315	2008-04-08	12
-84	val_84	2008-04-08	12
-28	val_28	2008-04-08	12
-37	val_37	2008-04-08	12
-448	val_448	2008-04-08	12
-152	val_152	2008-04-08	12
-348	val_348	2008-04-08	12
-307	val_307	2008-04-08	12
-194	val_194	2008-04-08	12
-414	val_414	2008-04-08	12
-477	val_477	2008-04-08	12
-222	val_222	2008-04-08	12
-126	val_126	2008-04-08	12
-90	val_90	2008-04-08	12
-169	val_169	2008-04-08	12
-403	val_403	2008-04-08	12
-400	val_400	2008-04-08	12
-200	val_200	2008-04-08	12
-97	val_97	2008-04-08	12
-238	val_238	2008-04-08	existing_value
-86	val_86	2008-04-08	existing_value
-311	val_311	2008-04-08	existing_value
-27	val_27	2008-04-08	existing_value
-165	val_165	2008-04-08	existing_value
-409	val_409	2008-04-08	existing_value
-255	val_255	2008-04-08	existing_value
-278	val_278	2008-04-08	existing_value
-98	val_98	2008-04-08	existing_value
-484	val_484	2008-04-08	existing_value
-265	val_265	2008-04-08	existing_value
-193	val_193	2008-04-08	existing_value
-401	val_401	2008-04-08	existing_value
-150	val_150	2008-04-08	existing_value
-273	val_273	2008-04-08	existing_value
-224	val_224	2008-04-08	existing_value
-369	val_369	2008-04-08	existing_value
-66	val_66	2008-04-08	existing_value
-128	val_128	2008-04-08	existing_value
-213	val_213	2008-04-08	existing_value
-146	val_146	2008-04-08	existing_value
-406	val_406	2008-04-08	existing_value
-429	val_429	2008-04-08	existing_value
-374	val_374	2008-04-08	existing_value
-152	val_152	2008-04-08	existing_value
-469	val_469	2008-04-08	existing_value
-145	val_145	2008-04-08	existing_value
-495	val_495	2008-04-08	existing_value
-37	val_37	2008-04-08	existing_value
-327	val_327	2008-04-08	existing_value
-281	val_281	2008-04-08	existing_value
-277	val_277	2008-04-08	existing_value
-209	val_209	2008-04-08	existing_value
-15	val_15	2008-04-08	existing_value
-82	val_82	2008-04-08	existing_value
-403	val_403	2008-04-08	existing_value
-166	val_166	2008-04-08	existing_value
-417	val_417	2008-04-08	existing_value
-430	val_430	2008-04-08	existing_value
-252	val_252	2008-04-08	existing_value
-292	val_292	2008-04-08	existing_value
-219	val_219	2008-04-08	existing_value
-287	val_287	2008-04-08	existing_value
-153	val_153	2008-04-08	existing_value
-193	val_193	2008-04-08	existing_value
-338	val_338	2008-04-08	existing_value
-446	val_446	2008-04-08	existing_value
-459	val_459	2008-04-08	existing_value
-394	val_394	2008-04-08	existing_value
-237	val_237	2008-04-08	existing_value
-482	val_482	2008-04-08	existing_value
-174	val_174	2008-04-08	existing_value
-413	val_413	2008-04-08	existing_value
-494	val_494	2008-04-08	existing_value
-207	val_207	2008-04-08	existing_value
-199	val_199	2008-04-08	existing_value
-466	val_466	2008-04-08	existing_value
-208	val_208	2008-04-08	existing_value
-174	val_174	2008-04-08	existing_value
-399	val_399	2008-04-08	existing_value
-396	val_396	2008-04-08	existing_value
-247	val_247	2008-04-08	existing_value
-417	val_417	2008-04-08	existing_value
-489	val_489	2008-04-08	existing_value
-162	val_162	2008-04-08	existing_value
-377	val_377	2008-04-08	existing_value
-397	val_397	2008-04-08	existing_value
-309	val_309	2008-04-08	existing_value
-365	val_365	2008-04-08	existing_value
-266	val_266	2008-04-08	existing_value
-439	val_439	2008-04-08	existing_value
-342	val_342	2008-04-08	existing_value
-367	val_367	2008-04-08	existing_value
-325	val_325	2008-04-08	existing_value
-167	val_167	2008-04-08	existing_value
-195	val_195	2008-04-08	existing_value
-475	val_475	2008-04-08	existing_value
-17	val_17	2008-04-08	existing_value
-113	val_113	2008-04-08	existing_value
-155	val_155	2008-04-08	existing_value
-203	val_203	2008-04-08	existing_value
-339	val_339	2008-04-08	existing_value
-0	val_0	2008-04-08	existing_value
-455	val_455	2008-04-08	existing_value
-128	val_128	2008-04-08	existing_value
-311	val_311	2008-04-08	existing_value
-316	val_316	2008-04-08	existing_value
-57	val_57	2008-04-08	existing_value
-302	val_302	2008-04-08	existing_value
-205	val_205	2008-04-08	existing_value
-149	val_149	2008-04-08	existing_value
-438	val_438	2008-04-08	existing_value
-345	val_345	2008-04-08	existing_value
-129	val_129	2008-04-08	existing_value
-170	val_170	2008-04-08	existing_value
-20	val_20	2008-04-08	existing_value
-489	val_489	2008-04-08	existing_value
-157	val_157	2008-04-08	existing_value
-378	val_378	2008-04-08	existing_value
-221	val_221	2008-04-08	existing_value
-92	val_92	2008-04-08	existing_value
-111	val_111	2008-04-08	existing_value
-47	val_47	2008-04-08	existing_value
-72	val_72	2008-04-08	existing_value
-4	val_4	2008-04-08	existing_value
-280	val_280	2008-04-08	existing_value
-35	val_35	2008-04-08	existing_value
-427	val_427	2008-04-08	existing_value
-277	val_277	2008-04-08	existing_value
-208	val_208	2008-04-08	existing_value
-356	val_356	2008-04-08	existing_value
-399	val_399	2008-04-08	existing_value
-169	val_169	2008-04-08	existing_value
-382	val_382	2008-04-08	existing_value
-498	val_498	2008-04-08	existing_value
-125	val_125	2008-04-08	existing_value
-386	val_386	2008-04-08	existing_value
-437	val_437	2008-04-08	existing_value
-469	val_469	2008-04-08	existing_value
-192	val_192	2008-04-08	existing_value
-286	val_286	2008-04-08	existing_value
-187	val_187	2008-04-08	existing_value
-176	val_176	2008-04-08	existing_value
-54	val_54	2008-04-08	existing_value
-459	val_459	2008-04-08	existing_value
-51	val_51	2008-04-08	existing_value
-138	val_138	2008-04-08	existing_value
-103	val_103	2008-04-08	existing_value
-239	val_239	2008-04-08	existing_value
-213	val_213	2008-04-08	existing_value
-216	val_216	2008-04-08	existing_value
-430	val_430	2008-04-08	existing_value
-278	val_278	2008-04-08	existing_value
-176	val_176	2008-04-08	existing_value
-289	val_289	2008-04-08	existing_value
-221	val_221	2008-04-08	existing_value
-65	val_65	2008-04-08	existing_value
-318	val_318	2008-04-08	existing_value
-332	val_332	2008-04-08	existing_value
-311	val_311	2008-04-08	existing_value
-275	val_275	2008-04-08	existing_value
-137	val_137	2008-04-08	existing_value
-241	val_241	2008-04-08	existing_value
-83	val_83	2008-04-08	existing_value
-333	val_333	2008-04-08	existing_value
-180	val_180	2008-04-08	existing_value
-284	val_284	2008-04-08	existing_value
-12	val_12	2008-04-08	existing_value
-230	val_230	2008-04-08	existing_value
-181	val_181	2008-04-08	existing_value
-67	val_67	2008-04-08	existing_value
-260	val_260	2008-04-08	existing_value
-404	val_404	2008-04-08	existing_value
-384	val_384	2008-04-08	existing_value
-489	val_489	2008-04-08	existing_value
-353	val_353	2008-04-08	existing_value
-373	val_373	2008-04-08	existing_value
-272	val_272	2008-04-08	existing_value
-138	val_138	2008-04-08	existing_value
-217	val_217	2008-04-08	existing_value
-84	val_84	2008-04-08	existing_value
-348	val_348	2008-04-08	existing_value
-466	val_466	2008-04-08	existing_value
-58	val_58	2008-04-08	existing_value
-8	val_8	2008-04-08	existing_value
-411	val_411	2008-04-08	existing_value
-230	val_230	2008-04-08	existing_value
-208	val_208	2008-04-08	existing_value
-348	val_348	2008-04-08	existing_value
-24	val_24	2008-04-08	existing_value
-463	val_463	2008-04-08	existing_value
-431	val_431	2008-04-08	existing_value
-179	val_179	2008-04-08	existing_value
-172	val_172	2008-04-08	existing_value
-42	val_42	2008-04-08	existing_value
-129	val_129	2008-04-08	existing_value
-158	val_158	2008-04-08	existing_value
-119	val_119	2008-04-08	existing_value
-496	val_496	2008-04-08	existing_value
-0	val_0	2008-04-08	existing_value
-322	val_322	2008-04-08	existing_value
-197	val_197	2008-04-08	existing_value
-468	val_468	2008-04-08	existing_value
-393	val_393	2008-04-08	existing_value
-454	val_454	2008-04-08	existing_value
-100	val_100	2008-04-08	existing_value
-298	val_298	2008-04-08	existing_value
-199	val_199	2008-04-08	existing_value
-191	val_191	2008-04-08	existing_value
-418	val_418	2008-04-08	existing_value
-96	val_96	2008-04-08	existing_value
-26	val_26	2008-04-08	existing_value
-165	val_165	2008-04-08	existing_value
-327	val_327	2008-04-08	existing_value
-230	val_230	2008-04-08	existing_value
-205	val_205	2008-04-08	existing_value
-120	val_120	2008-04-08	existing_value
-131	val_131	2008-04-08	existing_value
-51	val_51	2008-04-08	existing_value
-404	val_404	2008-04-08	existing_value
-43	val_43	2008-04-08	existing_value
-436	val_436	2008-04-08	existing_value
-156	val_156	2008-04-08	existing_value
-469	val_469	2008-04-08	existing_value
-468	val_468	2008-04-08	existing_value
-308	val_308	2008-04-08	existing_value
-95	val_95	2008-04-08	existing_value
-196	val_196	2008-04-08	existing_value
-288	val_288	2008-04-08	existing_value
-481	val_481	2008-04-08	existing_value
-457	val_457	2008-04-08	existing_value
-98	val_98	2008-04-08	existing_value
-282	val_282	2008-04-08	existing_value
-197	val_197	2008-04-08	existing_value
-187	val_187	2008-04-08	existing_value
-318	val_318	2008-04-08	existing_value
-318	val_318	2008-04-08	existing_value
-409	val_409	2008-04-08	existing_value
-470	val_470	2008-04-08	existing_value
-137	val_137	2008-04-08	existing_value
-369	val_369	2008-04-08	existing_value
-316	val_316	2008-04-08	existing_value
-169	val_169	2008-04-08	existing_value
-413	val_413	2008-04-08	existing_value
-85	val_85	2008-04-08	existing_value
-77	val_77	2008-04-08	existing_value
-0	val_0	2008-04-08	existing_value
-490	val_490	2008-04-08	existing_value
-87	val_87	2008-04-08	existing_value
-364	val_364	2008-04-08	existing_value
-179	val_179	2008-04-08	existing_value
-118	val_118	2008-04-08	existing_value
-134	val_134	2008-04-08	existing_value
-395	val_395	2008-04-08	existing_value
-282	val_282	2008-04-08	existing_value
-138	val_138	2008-04-08	existing_value
-238	val_238	2008-04-08	existing_value
-419	val_419	2008-04-08	existing_value
-15	val_15	2008-04-08	existing_value
-118	val_118	2008-04-08	existing_value
-72	val_72	2008-04-08	existing_value
-90	val_90	2008-04-08	existing_value
-307	val_307	2008-04-08	existing_value
-19	val_19	2008-04-08	existing_value
-435	val_435	2008-04-08	existing_value
-10	val_10	2008-04-08	existing_value
-277	val_277	2008-04-08	existing_value
-273	val_273	2008-04-08	existing_value
-306	val_306	2008-04-08	existing_value
-224	val_224	2008-04-08	existing_value
-309	val_309	2008-04-08	existing_value
-389	val_389	2008-04-08	existing_value
-327	val_327	2008-04-08	existing_value
-242	val_242	2008-04-08	existing_value
-369	val_369	2008-04-08	existing_value
-392	val_392	2008-04-08	existing_value
-272	val_272	2008-04-08	existing_value
-331	val_331	2008-04-08	existing_value
-401	val_401	2008-04-08	existing_value
-242	val_242	2008-04-08	existing_value
-452	val_452	2008-04-08	existing_value
-177	val_177	2008-04-08	existing_value
-226	val_226	2008-04-08	existing_value
-5	val_5	2008-04-08	existing_value
-497	val_497	2008-04-08	existing_value
-402	val_402	2008-04-08	existing_value
-396	val_396	2008-04-08	existing_value
-317	val_317	2008-04-08	existing_value
-395	val_395	2008-04-08	existing_value
-58	val_58	2008-04-08	existing_value
-35	val_35	2008-04-08	existing_value
-336	val_336	2008-04-08	existing_value
-95	val_95	2008-04-08	existing_value
-11	val_11	2008-04-08	existing_value
-168	val_168	2008-04-08	existing_value
-34	val_34	2008-04-08	existing_value
-229	val_229	2008-04-08	existing_value
-233	val_233	2008-04-08	existing_value
-143	val_143	2008-04-08	existing_value
-472	val_472	2008-04-08	existing_value
-322	val_322	2008-04-08	existing_value
-498	val_498	2008-04-08	existing_value
-160	val_160	2008-04-08	existing_value
-195	val_195	2008-04-08	existing_value
-42	val_42	2008-04-08	existing_value
-321	val_321	2008-04-08	existing_value
-430	val_430	2008-04-08	existing_value
-119	val_119	2008-04-08	existing_value
-489	val_489	2008-04-08	existing_value
-458	val_458	2008-04-08	existing_value
-78	val_78	2008-04-08	existing_value
-76	val_76	2008-04-08	existing_value
-41	val_41	2008-04-08	existing_value
-223	val_223	2008-04-08	existing_value
-492	val_492	2008-04-08	existing_value
-149	val_149	2008-04-08	existing_value
-449	val_449	2008-04-08	existing_value
-218	val_218	2008-04-08	existing_value
-228	val_228	2008-04-08	existing_value
-138	val_138	2008-04-08	existing_value
-453	val_453	2008-04-08	existing_value
-30	val_30	2008-04-08	existing_value
-209	val_209	2008-04-08	existing_value
-64	val_64	2008-04-08	existing_value
-468	val_468	2008-04-08	existing_value
-76	val_76	2008-04-08	existing_value
-74	val_74	2008-04-08	existing_value
-342	val_342	2008-04-08	existing_value
-69	val_69	2008-04-08	existing_value
-230	val_230	2008-04-08	existing_value
-33	val_33	2008-04-08	existing_value
-368	val_368	2008-04-08	existing_value
-103	val_103	2008-04-08	existing_value
-296	val_296	2008-04-08	existing_value
-113	val_113	2008-04-08	existing_value
-216	val_216	2008-04-08	existing_value
-367	val_367	2008-04-08	existing_value
-344	val_344	2008-04-08	existing_value
-167	val_167	2008-04-08	existing_value
-274	val_274	2008-04-08	existing_value
-219	val_219	2008-04-08	existing_value
-239	val_239	2008-04-08	existing_value
-485	val_485	2008-04-08	existing_value
-116	val_116	2008-04-08	existing_value
-223	val_223	2008-04-08	existing_value
-256	val_256	2008-04-08	existing_value
-263	val_263	2008-04-08	existing_value
-70	val_70	2008-04-08	existing_value
-487	val_487	2008-04-08	existing_value
-480	val_480	2008-04-08	existing_value
-401	val_401	2008-04-08	existing_value
-288	val_288	2008-04-08	existing_value
-191	val_191	2008-04-08	existing_value
-5	val_5	2008-04-08	existing_value
-244	val_244	2008-04-08	existing_value
-438	val_438	2008-04-08	existing_value
-128	val_128	2008-04-08	existing_value
-467	val_467	2008-04-08	existing_value
-432	val_432	2008-04-08	existing_value
-202	val_202	2008-04-08	existing_value
-316	val_316	2008-04-08	existing_value
-229	val_229	2008-04-08	existing_value
-469	val_469	2008-04-08	existing_value
-463	val_463	2008-04-08	existing_value
-280	val_280	2008-04-08	existing_value
-2	val_2	2008-04-08	existing_value
-35	val_35	2008-04-08	existing_value
-283	val_283	2008-04-08	existing_value
-331	val_331	2008-04-08	existing_value
-235	val_235	2008-04-08	existing_value
-80	val_80	2008-04-08	existing_value
-44	val_44	2008-04-08	existing_value
-193	val_193	2008-04-08	existing_value
-321	val_321	2008-04-08	existing_value
-335	val_335	2008-04-08	existing_value
-104	val_104	2008-04-08	existing_value
-466	val_466	2008-04-08	existing_value
-366	val_366	2008-04-08	existing_value
-175	val_175	2008-04-08	existing_value
-403	val_403	2008-04-08	existing_value
-483	val_483	2008-04-08	existing_value
-53	val_53	2008-04-08	existing_value
-105	val_105	2008-04-08	existing_value
-257	val_257	2008-04-08	existing_value
-406	val_406	2008-04-08	existing_value
-409	val_409	2008-04-08	existing_value
-190	val_190	2008-04-08	existing_value
-406	val_406	2008-04-08	existing_value
-401	val_401	2008-04-08	existing_value
-114	val_114	2008-04-08	existing_value
-258	val_258	2008-04-08	existing_value
-90	val_90	2008-04-08	existing_value
-203	val_203	2008-04-08	existing_value
-262	val_262	2008-04-08	existing_value
-348	val_348	2008-04-08	existing_value
-424	val_424	2008-04-08	existing_value
-12	val_12	2008-04-08	existing_value
-396	val_396	2008-04-08	existing_value
-201	val_201	2008-04-08	existing_value
-217	val_217	2008-04-08	existing_value
-164	val_164	2008-04-08	existing_value
-431	val_431	2008-04-08	existing_value
-454	val_454	2008-04-08	existing_value
-478	val_478	2008-04-08	existing_value
-298	val_298	2008-04-08	existing_value
-125	val_125	2008-04-08	existing_value
-431	val_431	2008-04-08	existing_value
-164	val_164	2008-04-08	existing_value
-424	val_424	2008-04-08	existing_value
-187	val_187	2008-04-08	existing_value
-382	val_382	2008-04-08	existing_value
-5	val_5	2008-04-08	existing_value
-70	val_70	2008-04-08	existing_value
-397	val_397	2008-04-08	existing_value
-480	val_480	2008-04-08	existing_value
-291	val_291	2008-04-08	existing_value
-24	val_24	2008-04-08	existing_value
-351	val_351	2008-04-08	existing_value
-255	val_255	2008-04-08	existing_value
-104	val_104	2008-04-08	existing_value
-70	val_70	2008-04-08	existing_value
-163	val_163	2008-04-08	existing_value
-438	val_438	2008-04-08	existing_value
-119	val_119	2008-04-08	existing_value
-414	val_414	2008-04-08	existing_value
-200	val_200	2008-04-08	existing_value
-491	val_491	2008-04-08	existing_value
-237	val_237	2008-04-08	existing_value
-439	val_439	2008-04-08	existing_value
-360	val_360	2008-04-08	existing_value
-248	val_248	2008-04-08	existing_value
-479	val_479	2008-04-08	existing_value
-305	val_305	2008-04-08	existing_value
-417	val_417	2008-04-08	existing_value
-199	val_199	2008-04-08	existing_value
-444	val_444	2008-04-08	existing_value
-120	val_120	2008-04-08	existing_value
-429	val_429	2008-04-08	existing_value
-169	val_169	2008-04-08	existing_value
-443	val_443	2008-04-08	existing_value
-323	val_323	2008-04-08	existing_value
-325	val_325	2008-04-08	existing_value
-277	val_277	2008-04-08	existing_value
-230	val_230	2008-04-08	existing_value
-478	val_478	2008-04-08	existing_value
-178	val_178	2008-04-08	existing_value
-468	val_468	2008-04-08	existing_value
-310	val_310	2008-04-08	existing_value
-317	val_317	2008-04-08	existing_value
-333	val_333	2008-04-08	existing_value
-493	val_493	2008-04-08	existing_value
-460	val_460	2008-04-08	existing_value
-207	val_207	2008-04-08	existing_value
-249	val_249	2008-04-08	existing_value
-265	val_265	2008-04-08	existing_value
-480	val_480	2008-04-08	existing_value
-83	val_83	2008-04-08	existing_value
-136	val_136	2008-04-08	existing_value
-353	val_353	2008-04-08	existing_value
-172	val_172	2008-04-08	existing_value
-214	val_214	2008-04-08	existing_value
-462	val_462	2008-04-08	existing_value
-233	val_233	2008-04-08	existing_value
-406	val_406	2008-04-08	existing_value
-133	val_133	2008-04-08	existing_value
-175	val_175	2008-04-08	existing_value
-189	val_189	2008-04-08	existing_value
-454	val_454	2008-04-08	existing_value
-375	val_375	2008-04-08	existing_value
-401	val_401	2008-04-08	existing_value
-421	val_421	2008-04-08	existing_value
-407	val_407	2008-04-08	existing_value
-384	val_384	2008-04-08	existing_value
-256	val_256	2008-04-08	existing_value
-26	val_26	2008-04-08	existing_value
-134	val_134	2008-04-08	existing_value
-67	val_67	2008-04-08	existing_value
-384	val_384	2008-04-08	existing_value
-379	val_379	2008-04-08	existing_value
-18	val_18	2008-04-08	existing_value
-462	val_462	2008-04-08	existing_value
-492	val_492	2008-04-08	existing_value
-100	val_100	2008-04-08	existing_value
-298	val_298	2008-04-08	existing_value
-9	val_9	2008-04-08	existing_value
-341	val_341	2008-04-08	existing_value
-498	val_498	2008-04-08	existing_value
-146	val_146	2008-04-08	existing_value
-458	val_458	2008-04-08	existing_value
-362	val_362	2008-04-08	existing_value
-186	val_186	2008-04-08	existing_value
-285	val_285	2008-04-08	existing_value
-348	val_348	2008-04-08	existing_value
-167	val_167	2008-04-08	existing_value
-18	val_18	2008-04-08	existing_value
-273	val_273	2008-04-08	existing_value
-183	val_183	2008-04-08	existing_value
-281	val_281	2008-04-08	existing_value
-344	val_344	2008-04-08	existing_value
-97	val_97	2008-04-08	existing_value
-469	val_469	2008-04-08	existing_value
-315	val_315	2008-04-08	existing_value
-84	val_84	2008-04-08	existing_value
-28	val_28	2008-04-08	existing_value
-37	val_37	2008-04-08	existing_value
-448	val_448	2008-04-08	existing_value
-152	val_152	2008-04-08	existing_value
-348	val_348	2008-04-08	existing_value
-307	val_307	2008-04-08	existing_value
-194	val_194	2008-04-08	existing_value
-414	val_414	2008-04-08	existing_value
-477	val_477	2008-04-08	existing_value
-222	val_222	2008-04-08	existing_value
-126	val_126	2008-04-08	existing_value
-90	val_90	2008-04-08	existing_value
-169	val_169	2008-04-08	existing_value
-403	val_403	2008-04-08	existing_value
-400	val_400	2008-04-08	existing_value
-200	val_200	2008-04-08	existing_value
-97	val_97	2008-04-08	existing_value
-238	val_238	2008-04-09	11
-86	val_86	2008-04-09	11
-311	val_311	2008-04-09	11
-27	val_27	2008-04-09	11
-165	val_165	2008-04-09	11
-409	val_409	2008-04-09	11
-255	val_255	2008-04-09	11
-278	val_278	2008-04-09	11
-98	val_98	2008-04-09	11
-484	val_484	2008-04-09	11
-265	val_265	2008-04-09	11
-193	val_193	2008-04-09	11
-401	val_401	2008-04-09	11
-150	val_150	2008-04-09	11
-273	val_273	2008-04-09	11
-224	val_224	2008-04-09	11
-369	val_369	2008-04-09	11
-66	val_66	2008-04-09	11
-128	val_128	2008-04-09	11
-213	val_213	2008-04-09	11
-146	val_146	2008-04-09	11
-406	val_406	2008-04-09	11
-429	val_429	2008-04-09	11
-374	val_374	2008-04-09	11
-152	val_152	2008-04-09	11
-469	val_469	2008-04-09	11
-145	val_145	2008-04-09	11
-495	val_495	2008-04-09	11
-37	val_37	2008-04-09	11
-327	val_327	2008-04-09	11
-281	val_281	2008-04-09	11
-277	val_277	2008-04-09	11
-209	val_209	2008-04-09	11
-15	val_15	2008-04-09	11
-82	val_82	2008-04-09	11
-403	val_403	2008-04-09	11
-166	val_166	2008-04-09	11
-417	val_417	2008-04-09	11
-430	val_430	2008-04-09	11
-252	val_252	2008-04-09	11
-292	val_292	2008-04-09	11
-219	val_219	2008-04-09	11
-287	val_287	2008-04-09	11
-153	val_153	2008-04-09	11
-193	val_193	2008-04-09	11
-338	val_338	2008-04-09	11
-446	val_446	2008-04-09	11
-459	val_459	2008-04-09	11
-394	val_394	2008-04-09	11
-237	val_237	2008-04-09	11
-482	val_482	2008-04-09	11
-174	val_174	2008-04-09	11
-413	val_413	2008-04-09	11
-494	val_494	2008-04-09	11
-207	val_207	2008-04-09	11
-199	val_199	2008-04-09	11
-466	val_466	2008-04-09	11
-208	val_208	2008-04-09	11
-174	val_174	2008-04-09	11
-399	val_399	2008-04-09	11
-396	val_396	2008-04-09	11
-247	val_247	2008-04-09	11
-417	val_417	2008-04-09	11
-489	val_489	2008-04-09	11
-162	val_162	2008-04-09	11
-377	val_377	2008-04-09	11
-397	val_397	2008-04-09	11
-309	val_309	2008-04-09	11
-365	val_365	2008-04-09	11
-266	val_266	2008-04-09	11
-439	val_439	2008-04-09	11
-342	val_342	2008-04-09	11
-367	val_367	2008-04-09	11
-325	val_325	2008-04-09	11
-167	val_167	2008-04-09	11
-195	val_195	2008-04-09	11
-475	val_475	2008-04-09	11
-17	val_17	2008-04-09	11
-113	val_113	2008-04-09	11
-155	val_155	2008-04-09	11
-203	val_203	2008-04-09	11
-339	val_339	2008-04-09	11
-0	val_0	2008-04-09	11
-455	val_455	2008-04-09	11
-128	val_128	2008-04-09	11
-311	val_311	2008-04-09	11
-316	val_316	2008-04-09	11
-57	val_57	2008-04-09	11
-302	val_302	2008-04-09	11
-205	val_205	2008-04-09	11
-149	val_149	2008-04-09	11
-438	val_438	2008-04-09	11
-345	val_345	2008-04-09	11
-129	val_129	2008-04-09	11
-170	val_170	2008-04-09	11
-20	val_20	2008-04-09	11
-489	val_489	2008-04-09	11
-157	val_157	2008-04-09	11
-378	val_378	2008-04-09	11
-221	val_221	2008-04-09	11
-92	val_92	2008-04-09	11
-111	val_111	2008-04-09	11
-47	val_47	2008-04-09	11
-72	val_72	2008-04-09	11
-4	val_4	2008-04-09	11
-280	val_280	2008-04-09	11
-35	val_35	2008-04-09	11
-427	val_427	2008-04-09	11
-277	val_277	2008-04-09	11
-208	val_208	2008-04-09	11
-356	val_356	2008-04-09	11
-399	val_399	2008-04-09	11
-169	val_169	2008-04-09	11
-382	val_382	2008-04-09	11
-498	val_498	2008-04-09	11
-125	val_125	2008-04-09	11
-386	val_386	2008-04-09	11
-437	val_437	2008-04-09	11
-469	val_469	2008-04-09	11
-192	val_192	2008-04-09	11
-286	val_286	2008-04-09	11
-187	val_187	2008-04-09	11
-176	val_176	2008-04-09	11
-54	val_54	2008-04-09	11
-459	val_459	2008-04-09	11
-51	val_51	2008-04-09	11
-138	val_138	2008-04-09	11
-103	val_103	2008-04-09	11
-239	val_239	2008-04-09	11
-213	val_213	2008-04-09	11
-216	val_216	2008-04-09	11
-430	val_430	2008-04-09	11
-278	val_278	2008-04-09	11
-176	val_176	2008-04-09	11
-289	val_289	2008-04-09	11
-221	val_221	2008-04-09	11
-65	val_65	2008-04-09	11
-318	val_318	2008-04-09	11
-332	val_332	2008-04-09	11
-311	val_311	2008-04-09	11
-275	val_275	2008-04-09	11
-137	val_137	2008-04-09	11
-241	val_241	2008-04-09	11
-83	val_83	2008-04-09	11
-333	val_333	2008-04-09	11
-180	val_180	2008-04-09	11
-284	val_284	2008-04-09	11
-12	val_12	2008-04-09	11
-230	val_230	2008-04-09	11
-181	val_181	2008-04-09	11
-67	val_67	2008-04-09	11
-260	val_260	2008-04-09	11
-404	val_404	2008-04-09	11
-384	val_384	2008-04-09	11
-489	val_489	2008-04-09	11
-353	val_353	2008-04-09	11
-373	val_373	2008-04-09	11
-272	val_272	2008-04-09	11
-138	val_138	2008-04-09	11
-217	val_217	2008-04-09	11
-84	val_84	2008-04-09	11
-348	val_348	2008-04-09	11
-466	val_466	2008-04-09	11
-58	val_58	2008-04-09	11
-8	val_8	2008-04-09	11
-411	val_411	2008-04-09	11
-230	val_230	2008-04-09	11
-208	val_208	2008-04-09	11
-348	val_348	2008-04-09	11
-24	val_24	2008-04-09	11
-463	val_463	2008-04-09	11
-431	val_431	2008-04-09	11
-179	val_179	2008-04-09	11
-172	val_172	2008-04-09	11
-42	val_42	2008-04-09	11
-129	val_129	2008-04-09	11
-158	val_158	2008-04-09	11
-119	val_119	2008-04-09	11
-496	val_496	2008-04-09	11
-0	val_0	2008-04-09	11
-322	val_322	2008-04-09	11
-197	val_197	2008-04-09	11
-468	val_468	2008-04-09	11
-393	val_393	2008-04-09	11
-454	val_454	2008-04-09	11
-100	val_100	2008-04-09	11
-298	val_298	2008-04-09	11
-199	val_199	2008-04-09	11
-191	val_191	2008-04-09	11
-418	val_418	2008-04-09	11
-96	val_96	2008-04-09	11
-26	val_26	2008-04-09	11
-165	val_165	2008-04-09	11
-327	val_327	2008-04-09	11
-230	val_230	2008-04-09	11
-205	val_205	2008-04-09	11
-120	val_120	2008-04-09	11
-131	val_131	2008-04-09	11
-51	val_51	2008-04-09	11
-404	val_404	2008-04-09	11
-43	val_43	2008-04-09	11
-436	val_436	2008-04-09	11
-156	val_156	2008-04-09	11
-469	val_469	2008-04-09	11
-468	val_468	2008-04-09	11
-308	val_308	2008-04-09	11
-95	val_95	2008-04-09	11
-196	val_196	2008-04-09	11
-288	val_288	2008-04-09	11
-481	val_481	2008-04-09	11
-457	val_457	2008-04-09	11
-98	val_98	2008-04-09	11
-282	val_282	2008-04-09	11
-197	val_197	2008-04-09	11
-187	val_187	2008-04-09	11
-318	val_318	2008-04-09	11
-318	val_318	2008-04-09	11
-409	val_409	2008-04-09	11
-470	val_470	2008-04-09	11
-137	val_137	2008-04-09	11
-369	val_369	2008-04-09	11
-316	val_316	2008-04-09	11
-169	val_169	2008-04-09	11
-413	val_413	2008-04-09	11
-85	val_85	2008-04-09	11
-77	val_77	2008-04-09	11
-0	val_0	2008-04-09	11
-490	val_490	2008-04-09	11
-87	val_87	2008-04-09	11
-364	val_364	2008-04-09	11
-179	val_179	2008-04-09	11
-118	val_118	2008-04-09	11
-134	val_134	2008-04-09	11
-395	val_395	2008-04-09	11
-282	val_282	2008-04-09	11
-138	val_138	2008-04-09	11
-238	val_238	2008-04-09	11
-419	val_419	2008-04-09	11
-15	val_15	2008-04-09	11
-118	val_118	2008-04-09	11
-72	val_72	2008-04-09	11
-90	val_90	2008-04-09	11
-307	val_307	2008-04-09	11
-19	val_19	2008-04-09	11
-435	val_435	2008-04-09	11
-10	val_10	2008-04-09	11
-277	val_277	2008-04-09	11
-273	val_273	2008-04-09	11
-306	val_306	2008-04-09	11
-224	val_224	2008-04-09	11
-309	val_309	2008-04-09	11
-389	val_389	2008-04-09	11
-327	val_327	2008-04-09	11
-242	val_242	2008-04-09	11
-369	val_369	2008-04-09	11
-392	val_392	2008-04-09	11
-272	val_272	2008-04-09	11
-331	val_331	2008-04-09	11
-401	val_401	2008-04-09	11
-242	val_242	2008-04-09	11
-452	val_452	2008-04-09	11
-177	val_177	2008-04-09	11
-226	val_226	2008-04-09	11
-5	val_5	2008-04-09	11
-497	val_497	2008-04-09	11
-402	val_402	2008-04-09	11
-396	val_396	2008-04-09	11
-317	val_317	2008-04-09	11
-395	val_395	2008-04-09	11
-58	val_58	2008-04-09	11
-35	val_35	2008-04-09	11
-336	val_336	2008-04-09	11
-95	val_95	2008-04-09	11
-11	val_11	2008-04-09	11
-168	val_168	2008-04-09	11
-34	val_34	2008-04-09	11
-229	val_229	2008-04-09	11
-233	val_233	2008-04-09	11
-143	val_143	2008-04-09	11
-472	val_472	2008-04-09	11
-322	val_322	2008-04-09	11
-498	val_498	2008-04-09	11
-160	val_160	2008-04-09	11
-195	val_195	2008-04-09	11
-42	val_42	2008-04-09	11
-321	val_321	2008-04-09	11
-430	val_430	2008-04-09	11
-119	val_119	2008-04-09	11
-489	val_489	2008-04-09	11
-458	val_458	2008-04-09	11
-78	val_78	2008-04-09	11
-76	val_76	2008-04-09	11
-41	val_41	2008-04-09	11
-223	val_223	2008-04-09	11
-492	val_492	2008-04-09	11
-149	val_149	2008-04-09	11
-449	val_449	2008-04-09	11
-218	val_218	2008-04-09	11
-228	val_228	2008-04-09	11
-138	val_138	2008-04-09	11
-453	val_453	2008-04-09	11
-30	val_30	2008-04-09	11
-209	val_209	2008-04-09	11
-64	val_64	2008-04-09	11
-468	val_468	2008-04-09	11
-76	val_76	2008-04-09	11
-74	val_74	2008-04-09	11
-342	val_342	2008-04-09	11
-69	val_69	2008-04-09	11
-230	val_230	2008-04-09	11
-33	val_33	2008-04-09	11
-368	val_368	2008-04-09	11
-103	val_103	2008-04-09	11
-296	val_296	2008-04-09	11
-113	val_113	2008-04-09	11
-216	val_216	2008-04-09	11
-367	val_367	2008-04-09	11
-344	val_344	2008-04-09	11
-167	val_167	2008-04-09	11
-274	val_274	2008-04-09	11
-219	val_219	2008-04-09	11
-239	val_239	2008-04-09	11
-485	val_485	2008-04-09	11
-116	val_116	2008-04-09	11
-223	val_223	2008-04-09	11
-256	val_256	2008-04-09	11
-263	val_263	2008-04-09	11
-70	val_70	2008-04-09	11
-487	val_487	2008-04-09	11
-480	val_480	2008-04-09	11
-401	val_401	2008-04-09	11
-288	val_288	2008-04-09	11
-191	val_191	2008-04-09	11
-5	val_5	2008-04-09	11
-244	val_244	2008-04-09	11
-438	val_438	2008-04-09	11
-128	val_128	2008-04-09	11
-467	val_467	2008-04-09	11
-432	val_432	2008-04-09	11
-202	val_202	2008-04-09	11
-316	val_316	2008-04-09	11
-229	val_229	2008-04-09	11
-469	val_469	2008-04-09	11
-463	val_463	2008-04-09	11
-280	val_280	2008-04-09	11
-2	val_2	2008-04-09	11
-35	val_35	2008-04-09	11
-283	val_283	2008-04-09	11
-331	val_331	2008-04-09	11
-235	val_235	2008-04-09	11
-80	val_80	2008-04-09	11
-44	val_44	2008-04-09	11
-193	val_193	2008-04-09	11
-321	val_321	2008-04-09	11
-335	val_335	2008-04-09	11
-104	val_104	2008-04-09	11
-466	val_466	2008-04-09	11
-366	val_366	2008-04-09	11
-175	val_175	2008-04-09	11
-403	val_403	2008-04-09	11
-483	val_483	2008-04-09	11
-53	val_53	2008-04-09	11
-105	val_105	2008-04-09	11
-257	val_257	2008-04-09	11
-406	val_406	2008-04-09	11
-409	val_409	2008-04-09	11
-190	val_190	2008-04-09	11
-406	val_406	2008-04-09	11
-401	val_401	2008-04-09	11
-114	val_114	2008-04-09	11
-258	val_258	2008-04-09	11
-90	val_90	2008-04-09	11
-203	val_203	2008-04-09	11
-262	val_262	2008-04-09	11
-348	val_348	2008-04-09	11
-424	val_424	2008-04-09	11
-12	val_12	2008-04-09	11
-396	val_396	2008-04-09	11
-201	val_201	2008-04-09	11
-217	val_217	2008-04-09	11
-164	val_164	2008-04-09	11
-431	val_431	2008-04-09	11
-454	val_454	2008-04-09	11
-478	val_478	2008-04-09	11
-298	val_298	2008-04-09	11
-125	val_125	2008-04-09	11
-431	val_431	2008-04-09	11
-164	val_164	2008-04-09	11
-424	val_424	2008-04-09	11
-187	val_187	2008-04-09	11
-382	val_382	2008-04-09	11
-5	val_5	2008-04-09	11
-70	val_70	2008-04-09	11
-397	val_397	2008-04-09	11
-480	val_480	2008-04-09	11
-291	val_291	2008-04-09	11
-24	val_24	2008-04-09	11
-351	val_351	2008-04-09	11
-255	val_255	2008-04-09	11
-104	val_104	2008-04-09	11
-70	val_70	2008-04-09	11
-163	val_163	2008-04-09	11
-438	val_438	2008-04-09	11
-119	val_119	2008-04-09	11
-414	val_414	2008-04-09	11
-200	val_200	2008-04-09	11
-491	val_491	2008-04-09	11
-237	val_237	2008-04-09	11
-439	val_439	2008-04-09	11
-360	val_360	2008-04-09	11
-248	val_248	2008-04-09	11
-479	val_479	2008-04-09	11
-305	val_305	2008-04-09	11
-417	val_417	2008-04-09	11
-199	val_199	2008-04-09	11
-444	val_444	2008-04-09	11
-120	val_120	2008-04-09	11
-429	val_429	2008-04-09	11
-169	val_169	2008-04-09	11
-443	val_443	2008-04-09	11
-323	val_323	2008-04-09	11
-325	val_325	2008-04-09	11
-277	val_277	2008-04-09	11
-230	val_230	2008-04-09	11
-478	val_478	2008-04-09	11
-178	val_178	2008-04-09	11
-468	val_468	2008-04-09	11
-310	val_310	2008-04-09	11
-317	val_317	2008-04-09	11
-333	val_333	2008-04-09	11
-493	val_493	2008-04-09	11
-460	val_460	2008-04-09	11
-207	val_207	2008-04-09	11
-249	val_249	2008-04-09	11
-265	val_265	2008-04-09	11
-480	val_480	2008-04-09	11
-83	val_83	2008-04-09	11
-136	val_136	2008-04-09	11
-353	val_353	2008-04-09	11
-172	val_172	2008-04-09	11
-214	val_214	2008-04-09	11
-462	val_462	2008-04-09	11
-233	val_233	2008-04-09	11
-406	val_406	2008-04-09	11
-133	val_133	2008-04-09	11
-175	val_175	2008-04-09	11
-189	val_189	2008-04-09	11
-454	val_454	2008-04-09	11
-375	val_375	2008-04-09	11
-401	val_401	2008-04-09	11
-421	val_421	2008-04-09	11
-407	val_407	2008-04-09	11
-384	val_384	2008-04-09	11
-256	val_256	2008-04-09	11
-26	val_26	2008-04-09	11
-134	val_134	2008-04-09	11
-67	val_67	2008-04-09	11
-384	val_384	2008-04-09	11
-379	val_379	2008-04-09	11
-18	val_18	2008-04-09	11
-462	val_462	2008-04-09	11
-492	val_492	2008-04-09	11
-100	val_100	2008-04-09	11
-298	val_298	2008-04-09	11
-9	val_9	2008-04-09	11
-341	val_341	2008-04-09	11
-498	val_498	2008-04-09	11
-146	val_146	2008-04-09	11
-458	val_458	2008-04-09	11
-362	val_362	2008-04-09	11
-186	val_186	2008-04-09	11
-285	val_285	2008-04-09	11
-348	val_348	2008-04-09	11
-167	val_167	2008-04-09	11
-18	val_18	2008-04-09	11
-273	val_273	2008-04-09	11
-183	val_183	2008-04-09	11
-281	val_281	2008-04-09	11
-344	val_344	2008-04-09	11
-97	val_97	2008-04-09	11
-469	val_469	2008-04-09	11
-315	val_315	2008-04-09	11
-84	val_84	2008-04-09	11
-28	val_28	2008-04-09	11
-37	val_37	2008-04-09	11
-448	val_448	2008-04-09	11
-152	val_152	2008-04-09	11
-348	val_348	2008-04-09	11
-307	val_307	2008-04-09	11
-194	val_194	2008-04-09	11
-414	val_414	2008-04-09	11
-477	val_477	2008-04-09	11
-222	val_222	2008-04-09	11
-126	val_126	2008-04-09	11
-90	val_90	2008-04-09	11
-169	val_169	2008-04-09	11
-403	val_403	2008-04-09	11
-400	val_400	2008-04-09	11
-200	val_200	2008-04-09	11
-97	val_97	2008-04-09	11
-238	val_238	2008-04-09	12
-86	val_86	2008-04-09	12
-311	val_311	2008-04-09	12
-27	val_27	2008-04-09	12
-165	val_165	2008-04-09	12
-409	val_409	2008-04-09	12
-255	val_255	2008-04-09	12
-278	val_278	2008-04-09	12
-98	val_98	2008-04-09	12
-484	val_484	2008-04-09	12
-265	val_265	2008-04-09	12
-193	val_193	2008-04-09	12
-401	val_401	2008-04-09	12
-150	val_150	2008-04-09	12
-273	val_273	2008-04-09	12
-224	val_224	2008-04-09	12
-369	val_369	2008-04-09	12
-66	val_66	2008-04-09	12
-128	val_128	2008-04-09	12
-213	val_213	2008-04-09	12
-146	val_146	2008-04-09	12
-406	val_406	2008-04-09	12
-429	val_429	2008-04-09	12
-374	val_374	2008-04-09	12
-152	val_152	2008-04-09	12
-469	val_469	2008-04-09	12
-145	val_145	2008-04-09	12
-495	val_495	2008-04-09	12
-37	val_37	2008-04-09	12
-327	val_327	2008-04-09	12
-281	val_281	2008-04-09	12
-277	val_277	2008-04-09	12
-209	val_209	2008-04-09	12
-15	val_15	2008-04-09	12
-82	val_82	2008-04-09	12
-403	val_403	2008-04-09	12
-166	val_166	2008-04-09	12
-417	val_417	2008-04-09	12
-430	val_430	2008-04-09	12
-252	val_252	2008-04-09	12
-292	val_292	2008-04-09	12
-219	val_219	2008-04-09	12
-287	val_287	2008-04-09	12
-153	val_153	2008-04-09	12
-193	val_193	2008-04-09	12
-338	val_338	2008-04-09	12
-446	val_446	2008-04-09	12
-459	val_459	2008-04-09	12
-394	val_394	2008-04-09	12
-237	val_237	2008-04-09	12
-482	val_482	2008-04-09	12
-174	val_174	2008-04-09	12
-413	val_413	2008-04-09	12
-494	val_494	2008-04-09	12
-207	val_207	2008-04-09	12
-199	val_199	2008-04-09	12
-466	val_466	2008-04-09	12
-208	val_208	2008-04-09	12
-174	val_174	2008-04-09	12
-399	val_399	2008-04-09	12
-396	val_396	2008-04-09	12
-247	val_247	2008-04-09	12
-417	val_417	2008-04-09	12
-489	val_489	2008-04-09	12
-162	val_162	2008-04-09	12
-377	val_377	2008-04-09	12
-397	val_397	2008-04-09	12
-309	val_309	2008-04-09	12
-365	val_365	2008-04-09	12
-266	val_266	2008-04-09	12
-439	val_439	2008-04-09	12
-342	val_342	2008-04-09	12
-367	val_367	2008-04-09	12
-325	val_325	2008-04-09	12
-167	val_167	2008-04-09	12
-195	val_195	2008-04-09	12
-475	val_475	2008-04-09	12
-17	val_17	2008-04-09	12
-113	val_113	2008-04-09	12
-155	val_155	2008-04-09	12
-203	val_203	2008-04-09	12
-339	val_339	2008-04-09	12
-0	val_0	2008-04-09	12
-455	val_455	2008-04-09	12
-128	val_128	2008-04-09	12
-311	val_311	2008-04-09	12
-316	val_316	2008-04-09	12
-57	val_57	2008-04-09	12
-302	val_302	2008-04-09	12
-205	val_205	2008-04-09	12
-149	val_149	2008-04-09	12
-438	val_438	2008-04-09	12
-345	val_345	2008-04-09	12
-129	val_129	2008-04-09	12
-170	val_170	2008-04-09	12
-20	val_20	2008-04-09	12
-489	val_489	2008-04-09	12
-157	val_157	2008-04-09	12
-378	val_378	2008-04-09	12
-221	val_221	2008-04-09	12
-92	val_92	2008-04-09	12
-111	val_111	2008-04-09	12
-47	val_47	2008-04-09	12
-72	val_72	2008-04-09	12
-4	val_4	2008-04-09	12
-280	val_280	2008-04-09	12
-35	val_35	2008-04-09	12
-427	val_427	2008-04-09	12
-277	val_277	2008-04-09	12
-208	val_208	2008-04-09	12
-356	val_356	2008-04-09	12
-399	val_399	2008-04-09	12
-169	val_169	2008-04-09	12
-382	val_382	2008-04-09	12
-498	val_498	2008-04-09	12
-125	val_125	2008-04-09	12
-386	val_386	2008-04-09	12
-437	val_437	2008-04-09	12
-469	val_469	2008-04-09	12
-192	val_192	2008-04-09	12
-286	val_286	2008-04-09	12
-187	val_187	2008-04-09	12
-176	val_176	2008-04-09	12
-54	val_54	2008-04-09	12
-459	val_459	2008-04-09	12
-51	val_51	2008-04-09	12
-138	val_138	2008-04-09	12
-103	val_103	2008-04-09	12
-239	val_239	2008-04-09	12
-213	val_213	2008-04-09	12
-216	val_216	2008-04-09	12
-430	val_430	2008-04-09	12
-278	val_278	2008-04-09	12
-176	val_176	2008-04-09	12
-289	val_289	2008-04-09	12
-221	val_221	2008-04-09	12
-65	val_65	2008-04-09	12
-318	val_318	2008-04-09	12
-332	val_332	2008-04-09	12
-311	val_311	2008-04-09	12
-275	val_275	2008-04-09	12
-137	val_137	2008-04-09	12
-241	val_241	2008-04-09	12
-83	val_83	2008-04-09	12
-333	val_333	2008-04-09	12
-180	val_180	2008-04-09	12
-284	val_284	2008-04-09	12
-12	val_12	2008-04-09	12
-230	val_230	2008-04-09	12
-181	val_181	2008-04-09	12
-67	val_67	2008-04-09	12
-260	val_260	2008-04-09	12
-404	val_404	2008-04-09	12
-384	val_384	2008-04-09	12
-489	val_489	2008-04-09	12
-353	val_353	2008-04-09	12
-373	val_373	2008-04-09	12
-272	val_272	2008-04-09	12
-138	val_138	2008-04-09	12
-217	val_217	2008-04-09	12
-84	val_84	2008-04-09	12
-348	val_348	2008-04-09	12
-466	val_466	2008-04-09	12
-58	val_58	2008-04-09	12
-8	val_8	2008-04-09	12
-411	val_411	2008-04-09	12
-230	val_230	2008-04-09	12
-208	val_208	2008-04-09	12
-348	val_348	2008-04-09	12
-24	val_24	2008-04-09	12
-463	val_463	2008-04-09	12
-431	val_431	2008-04-09	12
-179	val_179	2008-04-09	12
-172	val_172	2008-04-09	12
-42	val_42	2008-04-09	12
-129	val_129	2008-04-09	12
-158	val_158	2008-04-09	12
-119	val_119	2008-04-09	12
-496	val_496	2008-04-09	12
-0	val_0	2008-04-09	12
-322	val_322	2008-04-09	12
-197	val_197	2008-04-09	12
-468	val_468	2008-04-09	12
-393	val_393	2008-04-09	12
-454	val_454	2008-04-09	12
-100	val_100	2008-04-09	12
-298	val_298	2008-04-09	12
-199	val_199	2008-04-09	12
-191	val_191	2008-04-09	12
-418	val_418	2008-04-09	12
-96	val_96	2008-04-09	12
-26	val_26	2008-04-09	12
-165	val_165	2008-04-09	12
-327	val_327	2008-04-09	12
-230	val_230	2008-04-09	12
-205	val_205	2008-04-09	12
-120	val_120	2008-04-09	12
-131	val_131	2008-04-09	12
-51	val_51	2008-04-09	12
-404	val_404	2008-04-09	12
-43	val_43	2008-04-09	12
-436	val_436	2008-04-09	12
-156	val_156	2008-04-09	12
-469	val_469	2008-04-09	12
-468	val_468	2008-04-09	12
-308	val_308	2008-04-09	12
-95	val_95	2008-04-09	12
-196	val_196	2008-04-09	12
-288	val_288	2008-04-09	12
-481	val_481	2008-04-09	12
-457	val_457	2008-04-09	12
-98	val_98	2008-04-09	12
-282	val_282	2008-04-09	12
-197	val_197	2008-04-09	12
-187	val_187	2008-04-09	12
-318	val_318	2008-04-09	12
-318	val_318	2008-04-09	12
-409	val_409	2008-04-09	12
-470	val_470	2008-04-09	12
-137	val_137	2008-04-09	12
-369	val_369	2008-04-09	12
-316	val_316	2008-04-09	12
-169	val_169	2008-04-09	12
-413	val_413	2008-04-09	12
-85	val_85	2008-04-09	12
-77	val_77	2008-04-09	12
-0	val_0	2008-04-09	12
-490	val_490	2008-04-09	12
-87	val_87	2008-04-09	12
-364	val_364	2008-04-09	12
-179	val_179	2008-04-09	12
-118	val_118	2008-04-09	12
-134	val_134	2008-04-09	12
-395	val_395	2008-04-09	12
-282	val_282	2008-04-09	12
-138	val_138	2008-04-09	12
-238	val_238	2008-04-09	12
-419	val_419	2008-04-09	12
-15	val_15	2008-04-09	12
-118	val_118	2008-04-09	12
-72	val_72	2008-04-09	12
-90	val_90	2008-04-09	12
-307	val_307	2008-04-09	12
-19	val_19	2008-04-09	12
-435	val_435	2008-04-09	12
-10	val_10	2008-04-09	12
-277	val_277	2008-04-09	12
-273	val_273	2008-04-09	12
-306	val_306	2008-04-09	12
-224	val_224	2008-04-09	12
-309	val_309	2008-04-09	12
-389	val_389	2008-04-09	12
-327	val_327	2008-04-09	12
-242	val_242	2008-04-09	12
-369	val_369	2008-04-09	12
-392	val_392	2008-04-09	12
-272	val_272	2008-04-09	12
-331	val_331	2008-04-09	12
-401	val_401	2008-04-09	12
-242	val_242	2008-04-09	12
-452	val_452	2008-04-09	12
-177	val_177	2008-04-09	12
-226	val_226	2008-04-09	12
-5	val_5	2008-04-09	12
-497	val_497	2008-04-09	12
-402	val_402	2008-04-09	12
-396	val_396	2008-04-09	12
-317	val_317	2008-04-09	12
-395	val_395	2008-04-09	12
-58	val_58	2008-04-09	12
-35	val_35	2008-04-09	12
-336	val_336	2008-04-09	12
-95	val_95	2008-04-09	12
-11	val_11	2008-04-09	12
-168	val_168	2008-04-09	12
-34	val_34	2008-04-09	12
-229	val_229	2008-04-09	12
-233	val_233	2008-04-09	12
-143	val_143	2008-04-09	12
-472	val_472	2008-04-09	12
-322	val_322	2008-04-09	12
-498	val_498	2008-04-09	12
-160	val_160	2008-04-09	12
-195	val_195	2008-04-09	12
-42	val_42	2008-04-09	12
-321	val_321	2008-04-09	12
-430	val_430	2008-04-09	12
-119	val_119	2008-04-09	12
-489	val_489	2008-04-09	12
-458	val_458	2008-04-09	12
-78	val_78	2008-04-09	12
-76	val_76	2008-04-09	12
-41	val_41	2008-04-09	12
-223	val_223	2008-04-09	12
-492	val_492	2008-04-09	12
-149	val_149	2008-04-09	12
-449	val_449	2008-04-09	12
-218	val_218	2008-04-09	12
-228	val_228	2008-04-09	12
-138	val_138	2008-04-09	12
-453	val_453	2008-04-09	12
-30	val_30	2008-04-09	12
-209	val_209	2008-04-09	12
-64	val_64	2008-04-09	12
-468	val_468	2008-04-09	12
-76	val_76	2008-04-09	12
-74	val_74	2008-04-09	12
-342	val_342	2008-04-09	12
-69	val_69	2008-04-09	12
-230	val_230	2008-04-09	12
-33	val_33	2008-04-09	12
-368	val_368	2008-04-09	12
-103	val_103	2008-04-09	12
-296	val_296	2008-04-09	12
-113	val_113	2008-04-09	12
-216	val_216	2008-04-09	12
-367	val_367	2008-04-09	12
-344	val_344	2008-04-09	12
-167	val_167	2008-04-09	12
-274	val_274	2008-04-09	12
-219	val_219	2008-04-09	12
-239	val_239	2008-04-09	12
-485	val_485	2008-04-09	12
-116	val_116	2008-04-09	12
-223	val_223	2008-04-09	12
-256	val_256	2008-04-09	12
-263	val_263	2008-04-09	12
-70	val_70	2008-04-09	12
-487	val_487	2008-04-09	12
-480	val_480	2008-04-09	12
-401	val_401	2008-04-09	12
-288	val_288	2008-04-09	12
-191	val_191	2008-04-09	12
-5	val_5	2008-04-09	12
-244	val_244	2008-04-09	12
-438	val_438	2008-04-09	12
-128	val_128	2008-04-09	12
-467	val_467	2008-04-09	12
-432	val_432	2008-04-09	12
-202	val_202	2008-04-09	12
-316	val_316	2008-04-09	12
-229	val_229	2008-04-09	12
-469	val_469	2008-04-09	12
-463	val_463	2008-04-09	12
-280	val_280	2008-04-09	12
-2	val_2	2008-04-09	12
-35	val_35	2008-04-09	12
-283	val_283	2008-04-09	12
-331	val_331	2008-04-09	12
-235	val_235	2008-04-09	12
-80	val_80	2008-04-09	12
-44	val_44	2008-04-09	12
-193	val_193	2008-04-09	12
-321	val_321	2008-04-09	12
-335	val_335	2008-04-09	12
-104	val_104	2008-04-09	12
-466	val_466	2008-04-09	12
-366	val_366	2008-04-09	12
-175	val_175	2008-04-09	12
-403	val_403	2008-04-09	12
-483	val_483	2008-04-09	12
-53	val_53	2008-04-09	12
-105	val_105	2008-04-09	12
-257	val_257	2008-04-09	12
-406	val_406	2008-04-09	12
-409	val_409	2008-04-09	12
-190	val_190	2008-04-09	12
-406	val_406	2008-04-09	12
-401	val_401	2008-04-09	12
-114	val_114	2008-04-09	12
-258	val_258	2008-04-09	12
-90	val_90	2008-04-09	12
-203	val_203	2008-04-09	12
-262	val_262	2008-04-09	12
-348	val_348	2008-04-09	12
-424	val_424	2008-04-09	12
-12	val_12	2008-04-09	12
-396	val_396	2008-04-09	12
-201	val_201	2008-04-09	12
-217	val_217	2008-04-09	12
-164	val_164	2008-04-09	12
-431	val_431	2008-04-09	12
-454	val_454	2008-04-09	12
-478	val_478	2008-04-09	12
-298	val_298	2008-04-09	12
-125	val_125	2008-04-09	12
-431	val_431	2008-04-09	12
-164	val_164	2008-04-09	12
-424	val_424	2008-04-09	12
-187	val_187	2008-04-09	12
-382	val_382	2008-04-09	12
-5	val_5	2008-04-09	12
-70	val_70	2008-04-09	12
-397	val_397	2008-04-09	12
-480	val_480	2008-04-09	12
-291	val_291	2008-04-09	12
-24	val_24	2008-04-09	12
-351	val_351	2008-04-09	12
-255	val_255	2008-04-09	12
-104	val_104	2008-04-09	12
-70	val_70	2008-04-09	12
-163	val_163	2008-04-09	12
-438	val_438	2008-04-09	12
-119	val_119	2008-04-09	12
-414	val_414	2008-04-09	12
-200	val_200	2008-04-09	12
-491	val_491	2008-04-09	12
-237	val_237	2008-04-09	12
-439	val_439	2008-04-09	12
-360	val_360	2008-04-09	12
-248	val_248	2008-04-09	12
-479	val_479	2008-04-09	12
-305	val_305	2008-04-09	12
-417	val_417	2008-04-09	12
-199	val_199	2008-04-09	12
-444	val_444	2008-04-09	12
-120	val_120	2008-04-09	12
-429	val_429	2008-04-09	12
-169	val_169	2008-04-09	12
-443	val_443	2008-04-09	12
-323	val_323	2008-04-09	12
-325	val_325	2008-04-09	12
-277	val_277	2008-04-09	12
-230	val_230	2008-04-09	12
-478	val_478	2008-04-09	12
-178	val_178	2008-04-09	12
-468	val_468	2008-04-09	12
-310	val_310	2008-04-09	12
-317	val_317	2008-04-09	12
-333	val_333	2008-04-09	12
-493	val_493	2008-04-09	12
-460	val_460	2008-04-09	12
-207	val_207	2008-04-09	12
-249	val_249	2008-04-09	12
-265	val_265	2008-04-09	12
-480	val_480	2008-04-09	12
-83	val_83	2008-04-09	12
-136	val_136	2008-04-09	12
-353	val_353	2008-04-09	12
-172	val_172	2008-04-09	12
-214	val_214	2008-04-09	12
-462	val_462	2008-04-09	12
-233	val_233	2008-04-09	12
-406	val_406	2008-04-09	12
-133	val_133	2008-04-09	12
-175	val_175	2008-04-09	12
-189	val_189	2008-04-09	12
-454	val_454	2008-04-09	12
-375	val_375	2008-04-09	12
-401	val_401	2008-04-09	12
-421	val_421	2008-04-09	12
-407	val_407	2008-04-09	12
-384	val_384	2008-04-09	12
-256	val_256	2008-04-09	12
-26	val_26	2008-04-09	12
-134	val_134	2008-04-09	12
-67	val_67	2008-04-09	12
-384	val_384	2008-04-09	12
-379	val_379	2008-04-09	12
-18	val_18	2008-04-09	12
-462	val_462	2008-04-09	12
-492	val_492	2008-04-09	12
-100	val_100	2008-04-09	12
-298	val_298	2008-04-09	12
-9	val_9	2008-04-09	12
-341	val_341	2008-04-09	12
-498	val_498	2008-04-09	12
-146	val_146	2008-04-09	12
-458	val_458	2008-04-09	12
-362	val_362	2008-04-09	12
-186	val_186	2008-04-09	12
-285	val_285	2008-04-09	12
-348	val_348	2008-04-09	12
-167	val_167	2008-04-09	12
-18	val_18	2008-04-09	12
-273	val_273	2008-04-09	12
-183	val_183	2008-04-09	12
-281	val_281	2008-04-09	12
-344	val_344	2008-04-09	12
-97	val_97	2008-04-09	12
-469	val_469	2008-04-09	12
-315	val_315	2008-04-09	12
-84	val_84	2008-04-09	12
-28	val_28	2008-04-09	12
-37	val_37	2008-04-09	12
-448	val_448	2008-04-09	12
-152	val_152	2008-04-09	12
-348	val_348	2008-04-09	12
-307	val_307	2008-04-09	12
-194	val_194	2008-04-09	12
-414	val_414	2008-04-09	12
-477	val_477	2008-04-09	12
-222	val_222	2008-04-09	12
-126	val_126	2008-04-09	12
-90	val_90	2008-04-09	12
-169	val_169	2008-04-09	12
-403	val_403	2008-04-09	12
-400	val_400	2008-04-09	12
-200	val_200	2008-04-09	12
-97	val_97	2008-04-09	12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part4-11-24618a43c4656b72f81683b45708045e b/sql/hive/src/test/resources/golden/load_dyn_part4-11-24618a43c4656b72f81683b45708045e
new file mode 100644
index 0000000000000000000000000000000000000000..370744fed4f272bf350c9ebd375c55e0d403c86f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/load_dyn_part4-11-24618a43c4656b72f81683b45708045e
@@ -0,0 +1,2500 @@
+0	val_0	2008-04-08	11
+0	val_0	2008-04-08	11
+0	val_0	2008-04-08	11
+2	val_2	2008-04-08	11
+4	val_4	2008-04-08	11
+5	val_5	2008-04-08	11
+5	val_5	2008-04-08	11
+5	val_5	2008-04-08	11
+8	val_8	2008-04-08	11
+9	val_9	2008-04-08	11
+10	val_10	2008-04-08	11
+11	val_11	2008-04-08	11
+12	val_12	2008-04-08	11
+12	val_12	2008-04-08	11
+15	val_15	2008-04-08	11
+15	val_15	2008-04-08	11
+17	val_17	2008-04-08	11
+18	val_18	2008-04-08	11
+18	val_18	2008-04-08	11
+19	val_19	2008-04-08	11
+20	val_20	2008-04-08	11
+24	val_24	2008-04-08	11
+24	val_24	2008-04-08	11
+26	val_26	2008-04-08	11
+26	val_26	2008-04-08	11
+27	val_27	2008-04-08	11
+28	val_28	2008-04-08	11
+30	val_30	2008-04-08	11
+33	val_33	2008-04-08	11
+34	val_34	2008-04-08	11
+35	val_35	2008-04-08	11
+35	val_35	2008-04-08	11
+35	val_35	2008-04-08	11
+37	val_37	2008-04-08	11
+37	val_37	2008-04-08	11
+41	val_41	2008-04-08	11
+42	val_42	2008-04-08	11
+42	val_42	2008-04-08	11
+43	val_43	2008-04-08	11
+44	val_44	2008-04-08	11
+47	val_47	2008-04-08	11
+51	val_51	2008-04-08	11
+51	val_51	2008-04-08	11
+53	val_53	2008-04-08	11
+54	val_54	2008-04-08	11
+57	val_57	2008-04-08	11
+58	val_58	2008-04-08	11
+58	val_58	2008-04-08	11
+64	val_64	2008-04-08	11
+65	val_65	2008-04-08	11
+66	val_66	2008-04-08	11
+67	val_67	2008-04-08	11
+67	val_67	2008-04-08	11
+69	val_69	2008-04-08	11
+70	val_70	2008-04-08	11
+70	val_70	2008-04-08	11
+70	val_70	2008-04-08	11
+72	val_72	2008-04-08	11
+72	val_72	2008-04-08	11
+74	val_74	2008-04-08	11
+76	val_76	2008-04-08	11
+76	val_76	2008-04-08	11
+77	val_77	2008-04-08	11
+78	val_78	2008-04-08	11
+80	val_80	2008-04-08	11
+82	val_82	2008-04-08	11
+83	val_83	2008-04-08	11
+83	val_83	2008-04-08	11
+84	val_84	2008-04-08	11
+84	val_84	2008-04-08	11
+85	val_85	2008-04-08	11
+86	val_86	2008-04-08	11
+87	val_87	2008-04-08	11
+90	val_90	2008-04-08	11
+90	val_90	2008-04-08	11
+90	val_90	2008-04-08	11
+92	val_92	2008-04-08	11
+95	val_95	2008-04-08	11
+95	val_95	2008-04-08	11
+96	val_96	2008-04-08	11
+97	val_97	2008-04-08	11
+97	val_97	2008-04-08	11
+98	val_98	2008-04-08	11
+98	val_98	2008-04-08	11
+100	val_100	2008-04-08	11
+100	val_100	2008-04-08	11
+103	val_103	2008-04-08	11
+103	val_103	2008-04-08	11
+104	val_104	2008-04-08	11
+104	val_104	2008-04-08	11
+105	val_105	2008-04-08	11
+111	val_111	2008-04-08	11
+113	val_113	2008-04-08	11
+113	val_113	2008-04-08	11
+114	val_114	2008-04-08	11
+116	val_116	2008-04-08	11
+118	val_118	2008-04-08	11
+118	val_118	2008-04-08	11
+119	val_119	2008-04-08	11
+119	val_119	2008-04-08	11
+119	val_119	2008-04-08	11
+120	val_120	2008-04-08	11
+120	val_120	2008-04-08	11
+125	val_125	2008-04-08	11
+125	val_125	2008-04-08	11
+126	val_126	2008-04-08	11
+128	val_128	2008-04-08	11
+128	val_128	2008-04-08	11
+128	val_128	2008-04-08	11
+129	val_129	2008-04-08	11
+129	val_129	2008-04-08	11
+131	val_131	2008-04-08	11
+133	val_133	2008-04-08	11
+134	val_134	2008-04-08	11
+134	val_134	2008-04-08	11
+136	val_136	2008-04-08	11
+137	val_137	2008-04-08	11
+137	val_137	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+143	val_143	2008-04-08	11
+145	val_145	2008-04-08	11
+146	val_146	2008-04-08	11
+146	val_146	2008-04-08	11
+149	val_149	2008-04-08	11
+149	val_149	2008-04-08	11
+150	val_150	2008-04-08	11
+152	val_152	2008-04-08	11
+152	val_152	2008-04-08	11
+153	val_153	2008-04-08	11
+155	val_155	2008-04-08	11
+156	val_156	2008-04-08	11
+157	val_157	2008-04-08	11
+158	val_158	2008-04-08	11
+160	val_160	2008-04-08	11
+162	val_162	2008-04-08	11
+163	val_163	2008-04-08	11
+164	val_164	2008-04-08	11
+164	val_164	2008-04-08	11
+165	val_165	2008-04-08	11
+165	val_165	2008-04-08	11
+166	val_166	2008-04-08	11
+167	val_167	2008-04-08	11
+167	val_167	2008-04-08	11
+167	val_167	2008-04-08	11
+168	val_168	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+170	val_170	2008-04-08	11
+172	val_172	2008-04-08	11
+172	val_172	2008-04-08	11
+174	val_174	2008-04-08	11
+174	val_174	2008-04-08	11
+175	val_175	2008-04-08	11
+175	val_175	2008-04-08	11
+176	val_176	2008-04-08	11
+176	val_176	2008-04-08	11
+177	val_177	2008-04-08	11
+178	val_178	2008-04-08	11
+179	val_179	2008-04-08	11
+179	val_179	2008-04-08	11
+180	val_180	2008-04-08	11
+181	val_181	2008-04-08	11
+183	val_183	2008-04-08	11
+186	val_186	2008-04-08	11
+187	val_187	2008-04-08	11
+187	val_187	2008-04-08	11
+187	val_187	2008-04-08	11
+189	val_189	2008-04-08	11
+190	val_190	2008-04-08	11
+191	val_191	2008-04-08	11
+191	val_191	2008-04-08	11
+192	val_192	2008-04-08	11
+193	val_193	2008-04-08	11
+193	val_193	2008-04-08	11
+193	val_193	2008-04-08	11
+194	val_194	2008-04-08	11
+195	val_195	2008-04-08	11
+195	val_195	2008-04-08	11
+196	val_196	2008-04-08	11
+197	val_197	2008-04-08	11
+197	val_197	2008-04-08	11
+199	val_199	2008-04-08	11
+199	val_199	2008-04-08	11
+199	val_199	2008-04-08	11
+200	val_200	2008-04-08	11
+200	val_200	2008-04-08	11
+201	val_201	2008-04-08	11
+202	val_202	2008-04-08	11
+203	val_203	2008-04-08	11
+203	val_203	2008-04-08	11
+205	val_205	2008-04-08	11
+205	val_205	2008-04-08	11
+207	val_207	2008-04-08	11
+207	val_207	2008-04-08	11
+208	val_208	2008-04-08	11
+208	val_208	2008-04-08	11
+208	val_208	2008-04-08	11
+209	val_209	2008-04-08	11
+209	val_209	2008-04-08	11
+213	val_213	2008-04-08	11
+213	val_213	2008-04-08	11
+214	val_214	2008-04-08	11
+216	val_216	2008-04-08	11
+216	val_216	2008-04-08	11
+217	val_217	2008-04-08	11
+217	val_217	2008-04-08	11
+218	val_218	2008-04-08	11
+219	val_219	2008-04-08	11
+219	val_219	2008-04-08	11
+221	val_221	2008-04-08	11
+221	val_221	2008-04-08	11
+222	val_222	2008-04-08	11
+223	val_223	2008-04-08	11
+223	val_223	2008-04-08	11
+224	val_224	2008-04-08	11
+224	val_224	2008-04-08	11
+226	val_226	2008-04-08	11
+228	val_228	2008-04-08	11
+229	val_229	2008-04-08	11
+229	val_229	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+233	val_233	2008-04-08	11
+233	val_233	2008-04-08	11
+235	val_235	2008-04-08	11
+237	val_237	2008-04-08	11
+237	val_237	2008-04-08	11
+238	val_238	2008-04-08	11
+238	val_238	2008-04-08	11
+239	val_239	2008-04-08	11
+239	val_239	2008-04-08	11
+241	val_241	2008-04-08	11
+242	val_242	2008-04-08	11
+242	val_242	2008-04-08	11
+244	val_244	2008-04-08	11
+247	val_247	2008-04-08	11
+248	val_248	2008-04-08	11
+249	val_249	2008-04-08	11
+252	val_252	2008-04-08	11
+255	val_255	2008-04-08	11
+255	val_255	2008-04-08	11
+256	val_256	2008-04-08	11
+256	val_256	2008-04-08	11
+257	val_257	2008-04-08	11
+258	val_258	2008-04-08	11
+260	val_260	2008-04-08	11
+262	val_262	2008-04-08	11
+263	val_263	2008-04-08	11
+265	val_265	2008-04-08	11
+265	val_265	2008-04-08	11
+266	val_266	2008-04-08	11
+272	val_272	2008-04-08	11
+272	val_272	2008-04-08	11
+273	val_273	2008-04-08	11
+273	val_273	2008-04-08	11
+273	val_273	2008-04-08	11
+274	val_274	2008-04-08	11
+275	val_275	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+278	val_278	2008-04-08	11
+278	val_278	2008-04-08	11
+280	val_280	2008-04-08	11
+280	val_280	2008-04-08	11
+281	val_281	2008-04-08	11
+281	val_281	2008-04-08	11
+282	val_282	2008-04-08	11
+282	val_282	2008-04-08	11
+283	val_283	2008-04-08	11
+284	val_284	2008-04-08	11
+285	val_285	2008-04-08	11
+286	val_286	2008-04-08	11
+287	val_287	2008-04-08	11
+288	val_288	2008-04-08	11
+288	val_288	2008-04-08	11
+289	val_289	2008-04-08	11
+291	val_291	2008-04-08	11
+292	val_292	2008-04-08	11
+296	val_296	2008-04-08	11
+298	val_298	2008-04-08	11
+298	val_298	2008-04-08	11
+298	val_298	2008-04-08	11
+302	val_302	2008-04-08	11
+305	val_305	2008-04-08	11
+306	val_306	2008-04-08	11
+307	val_307	2008-04-08	11
+307	val_307	2008-04-08	11
+308	val_308	2008-04-08	11
+309	val_309	2008-04-08	11
+309	val_309	2008-04-08	11
+310	val_310	2008-04-08	11
+311	val_311	2008-04-08	11
+311	val_311	2008-04-08	11
+311	val_311	2008-04-08	11
+315	val_315	2008-04-08	11
+316	val_316	2008-04-08	11
+316	val_316	2008-04-08	11
+316	val_316	2008-04-08	11
+317	val_317	2008-04-08	11
+317	val_317	2008-04-08	11
+318	val_318	2008-04-08	11
+318	val_318	2008-04-08	11
+318	val_318	2008-04-08	11
+321	val_321	2008-04-08	11
+321	val_321	2008-04-08	11
+322	val_322	2008-04-08	11
+322	val_322	2008-04-08	11
+323	val_323	2008-04-08	11
+325	val_325	2008-04-08	11
+325	val_325	2008-04-08	11
+327	val_327	2008-04-08	11
+327	val_327	2008-04-08	11
+327	val_327	2008-04-08	11
+331	val_331	2008-04-08	11
+331	val_331	2008-04-08	11
+332	val_332	2008-04-08	11
+333	val_333	2008-04-08	11
+333	val_333	2008-04-08	11
+335	val_335	2008-04-08	11
+336	val_336	2008-04-08	11
+338	val_338	2008-04-08	11
+339	val_339	2008-04-08	11
+341	val_341	2008-04-08	11
+342	val_342	2008-04-08	11
+342	val_342	2008-04-08	11
+344	val_344	2008-04-08	11
+344	val_344	2008-04-08	11
+345	val_345	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+351	val_351	2008-04-08	11
+353	val_353	2008-04-08	11
+353	val_353	2008-04-08	11
+356	val_356	2008-04-08	11
+360	val_360	2008-04-08	11
+362	val_362	2008-04-08	11
+364	val_364	2008-04-08	11
+365	val_365	2008-04-08	11
+366	val_366	2008-04-08	11
+367	val_367	2008-04-08	11
+367	val_367	2008-04-08	11
+368	val_368	2008-04-08	11
+369	val_369	2008-04-08	11
+369	val_369	2008-04-08	11
+369	val_369	2008-04-08	11
+373	val_373	2008-04-08	11
+374	val_374	2008-04-08	11
+375	val_375	2008-04-08	11
+377	val_377	2008-04-08	11
+378	val_378	2008-04-08	11
+379	val_379	2008-04-08	11
+382	val_382	2008-04-08	11
+382	val_382	2008-04-08	11
+384	val_384	2008-04-08	11
+384	val_384	2008-04-08	11
+384	val_384	2008-04-08	11
+386	val_386	2008-04-08	11
+389	val_389	2008-04-08	11
+392	val_392	2008-04-08	11
+393	val_393	2008-04-08	11
+394	val_394	2008-04-08	11
+395	val_395	2008-04-08	11
+395	val_395	2008-04-08	11
+396	val_396	2008-04-08	11
+396	val_396	2008-04-08	11
+396	val_396	2008-04-08	11
+397	val_397	2008-04-08	11
+397	val_397	2008-04-08	11
+399	val_399	2008-04-08	11
+399	val_399	2008-04-08	11
+400	val_400	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+402	val_402	2008-04-08	11
+403	val_403	2008-04-08	11
+403	val_403	2008-04-08	11
+403	val_403	2008-04-08	11
+404	val_404	2008-04-08	11
+404	val_404	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+407	val_407	2008-04-08	11
+409	val_409	2008-04-08	11
+409	val_409	2008-04-08	11
+409	val_409	2008-04-08	11
+411	val_411	2008-04-08	11
+413	val_413	2008-04-08	11
+413	val_413	2008-04-08	11
+414	val_414	2008-04-08	11
+414	val_414	2008-04-08	11
+417	val_417	2008-04-08	11
+417	val_417	2008-04-08	11
+417	val_417	2008-04-08	11
+418	val_418	2008-04-08	11
+419	val_419	2008-04-08	11
+421	val_421	2008-04-08	11
+424	val_424	2008-04-08	11
+424	val_424	2008-04-08	11
+427	val_427	2008-04-08	11
+429	val_429	2008-04-08	11
+429	val_429	2008-04-08	11
+430	val_430	2008-04-08	11
+430	val_430	2008-04-08	11
+430	val_430	2008-04-08	11
+431	val_431	2008-04-08	11
+431	val_431	2008-04-08	11
+431	val_431	2008-04-08	11
+432	val_432	2008-04-08	11
+435	val_435	2008-04-08	11
+436	val_436	2008-04-08	11
+437	val_437	2008-04-08	11
+438	val_438	2008-04-08	11
+438	val_438	2008-04-08	11
+438	val_438	2008-04-08	11
+439	val_439	2008-04-08	11
+439	val_439	2008-04-08	11
+443	val_443	2008-04-08	11
+444	val_444	2008-04-08	11
+446	val_446	2008-04-08	11
+448	val_448	2008-04-08	11
+449	val_449	2008-04-08	11
+452	val_452	2008-04-08	11
+453	val_453	2008-04-08	11
+454	val_454	2008-04-08	11
+454	val_454	2008-04-08	11
+454	val_454	2008-04-08	11
+455	val_455	2008-04-08	11
+457	val_457	2008-04-08	11
+458	val_458	2008-04-08	11
+458	val_458	2008-04-08	11
+459	val_459	2008-04-08	11
+459	val_459	2008-04-08	11
+460	val_460	2008-04-08	11
+462	val_462	2008-04-08	11
+462	val_462	2008-04-08	11
+463	val_463	2008-04-08	11
+463	val_463	2008-04-08	11
+466	val_466	2008-04-08	11
+466	val_466	2008-04-08	11
+466	val_466	2008-04-08	11
+467	val_467	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+470	val_470	2008-04-08	11
+472	val_472	2008-04-08	11
+475	val_475	2008-04-08	11
+477	val_477	2008-04-08	11
+478	val_478	2008-04-08	11
+478	val_478	2008-04-08	11
+479	val_479	2008-04-08	11
+480	val_480	2008-04-08	11
+480	val_480	2008-04-08	11
+480	val_480	2008-04-08	11
+481	val_481	2008-04-08	11
+482	val_482	2008-04-08	11
+483	val_483	2008-04-08	11
+484	val_484	2008-04-08	11
+485	val_485	2008-04-08	11
+487	val_487	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+490	val_490	2008-04-08	11
+491	val_491	2008-04-08	11
+492	val_492	2008-04-08	11
+492	val_492	2008-04-08	11
+493	val_493	2008-04-08	11
+494	val_494	2008-04-08	11
+495	val_495	2008-04-08	11
+496	val_496	2008-04-08	11
+497	val_497	2008-04-08	11
+498	val_498	2008-04-08	11
+498	val_498	2008-04-08	11
+498	val_498	2008-04-08	11
+0	val_0	2008-04-08	12
+0	val_0	2008-04-08	12
+0	val_0	2008-04-08	12
+2	val_2	2008-04-08	12
+4	val_4	2008-04-08	12
+5	val_5	2008-04-08	12
+5	val_5	2008-04-08	12
+5	val_5	2008-04-08	12
+8	val_8	2008-04-08	12
+9	val_9	2008-04-08	12
+10	val_10	2008-04-08	12
+11	val_11	2008-04-08	12
+12	val_12	2008-04-08	12
+12	val_12	2008-04-08	12
+15	val_15	2008-04-08	12
+15	val_15	2008-04-08	12
+17	val_17	2008-04-08	12
+18	val_18	2008-04-08	12
+18	val_18	2008-04-08	12
+19	val_19	2008-04-08	12
+20	val_20	2008-04-08	12
+24	val_24	2008-04-08	12
+24	val_24	2008-04-08	12
+26	val_26	2008-04-08	12
+26	val_26	2008-04-08	12
+27	val_27	2008-04-08	12
+28	val_28	2008-04-08	12
+30	val_30	2008-04-08	12
+33	val_33	2008-04-08	12
+34	val_34	2008-04-08	12
+35	val_35	2008-04-08	12
+35	val_35	2008-04-08	12
+35	val_35	2008-04-08	12
+37	val_37	2008-04-08	12
+37	val_37	2008-04-08	12
+41	val_41	2008-04-08	12
+42	val_42	2008-04-08	12
+42	val_42	2008-04-08	12
+43	val_43	2008-04-08	12
+44	val_44	2008-04-08	12
+47	val_47	2008-04-08	12
+51	val_51	2008-04-08	12
+51	val_51	2008-04-08	12
+53	val_53	2008-04-08	12
+54	val_54	2008-04-08	12
+57	val_57	2008-04-08	12
+58	val_58	2008-04-08	12
+58	val_58	2008-04-08	12
+64	val_64	2008-04-08	12
+65	val_65	2008-04-08	12
+66	val_66	2008-04-08	12
+67	val_67	2008-04-08	12
+67	val_67	2008-04-08	12
+69	val_69	2008-04-08	12
+70	val_70	2008-04-08	12
+70	val_70	2008-04-08	12
+70	val_70	2008-04-08	12
+72	val_72	2008-04-08	12
+72	val_72	2008-04-08	12
+74	val_74	2008-04-08	12
+76	val_76	2008-04-08	12
+76	val_76	2008-04-08	12
+77	val_77	2008-04-08	12
+78	val_78	2008-04-08	12
+80	val_80	2008-04-08	12
+82	val_82	2008-04-08	12
+83	val_83	2008-04-08	12
+83	val_83	2008-04-08	12
+84	val_84	2008-04-08	12
+84	val_84	2008-04-08	12
+85	val_85	2008-04-08	12
+86	val_86	2008-04-08	12
+87	val_87	2008-04-08	12
+90	val_90	2008-04-08	12
+90	val_90	2008-04-08	12
+90	val_90	2008-04-08	12
+92	val_92	2008-04-08	12
+95	val_95	2008-04-08	12
+95	val_95	2008-04-08	12
+96	val_96	2008-04-08	12
+97	val_97	2008-04-08	12
+97	val_97	2008-04-08	12
+98	val_98	2008-04-08	12
+98	val_98	2008-04-08	12
+100	val_100	2008-04-08	12
+100	val_100	2008-04-08	12
+103	val_103	2008-04-08	12
+103	val_103	2008-04-08	12
+104	val_104	2008-04-08	12
+104	val_104	2008-04-08	12
+105	val_105	2008-04-08	12
+111	val_111	2008-04-08	12
+113	val_113	2008-04-08	12
+113	val_113	2008-04-08	12
+114	val_114	2008-04-08	12
+116	val_116	2008-04-08	12
+118	val_118	2008-04-08	12
+118	val_118	2008-04-08	12
+119	val_119	2008-04-08	12
+119	val_119	2008-04-08	12
+119	val_119	2008-04-08	12
+120	val_120	2008-04-08	12
+120	val_120	2008-04-08	12
+125	val_125	2008-04-08	12
+125	val_125	2008-04-08	12
+126	val_126	2008-04-08	12
+128	val_128	2008-04-08	12
+128	val_128	2008-04-08	12
+128	val_128	2008-04-08	12
+129	val_129	2008-04-08	12
+129	val_129	2008-04-08	12
+131	val_131	2008-04-08	12
+133	val_133	2008-04-08	12
+134	val_134	2008-04-08	12
+134	val_134	2008-04-08	12
+136	val_136	2008-04-08	12
+137	val_137	2008-04-08	12
+137	val_137	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+143	val_143	2008-04-08	12
+145	val_145	2008-04-08	12
+146	val_146	2008-04-08	12
+146	val_146	2008-04-08	12
+149	val_149	2008-04-08	12
+149	val_149	2008-04-08	12
+150	val_150	2008-04-08	12
+152	val_152	2008-04-08	12
+152	val_152	2008-04-08	12
+153	val_153	2008-04-08	12
+155	val_155	2008-04-08	12
+156	val_156	2008-04-08	12
+157	val_157	2008-04-08	12
+158	val_158	2008-04-08	12
+160	val_160	2008-04-08	12
+162	val_162	2008-04-08	12
+163	val_163	2008-04-08	12
+164	val_164	2008-04-08	12
+164	val_164	2008-04-08	12
+165	val_165	2008-04-08	12
+165	val_165	2008-04-08	12
+166	val_166	2008-04-08	12
+167	val_167	2008-04-08	12
+167	val_167	2008-04-08	12
+167	val_167	2008-04-08	12
+168	val_168	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+170	val_170	2008-04-08	12
+172	val_172	2008-04-08	12
+172	val_172	2008-04-08	12
+174	val_174	2008-04-08	12
+174	val_174	2008-04-08	12
+175	val_175	2008-04-08	12
+175	val_175	2008-04-08	12
+176	val_176	2008-04-08	12
+176	val_176	2008-04-08	12
+177	val_177	2008-04-08	12
+178	val_178	2008-04-08	12
+179	val_179	2008-04-08	12
+179	val_179	2008-04-08	12
+180	val_180	2008-04-08	12
+181	val_181	2008-04-08	12
+183	val_183	2008-04-08	12
+186	val_186	2008-04-08	12
+187	val_187	2008-04-08	12
+187	val_187	2008-04-08	12
+187	val_187	2008-04-08	12
+189	val_189	2008-04-08	12
+190	val_190	2008-04-08	12
+191	val_191	2008-04-08	12
+191	val_191	2008-04-08	12
+192	val_192	2008-04-08	12
+193	val_193	2008-04-08	12
+193	val_193	2008-04-08	12
+193	val_193	2008-04-08	12
+194	val_194	2008-04-08	12
+195	val_195	2008-04-08	12
+195	val_195	2008-04-08	12
+196	val_196	2008-04-08	12
+197	val_197	2008-04-08	12
+197	val_197	2008-04-08	12
+199	val_199	2008-04-08	12
+199	val_199	2008-04-08	12
+199	val_199	2008-04-08	12
+200	val_200	2008-04-08	12
+200	val_200	2008-04-08	12
+201	val_201	2008-04-08	12
+202	val_202	2008-04-08	12
+203	val_203	2008-04-08	12
+203	val_203	2008-04-08	12
+205	val_205	2008-04-08	12
+205	val_205	2008-04-08	12
+207	val_207	2008-04-08	12
+207	val_207	2008-04-08	12
+208	val_208	2008-04-08	12
+208	val_208	2008-04-08	12
+208	val_208	2008-04-08	12
+209	val_209	2008-04-08	12
+209	val_209	2008-04-08	12
+213	val_213	2008-04-08	12
+213	val_213	2008-04-08	12
+214	val_214	2008-04-08	12
+216	val_216	2008-04-08	12
+216	val_216	2008-04-08	12
+217	val_217	2008-04-08	12
+217	val_217	2008-04-08	12
+218	val_218	2008-04-08	12
+219	val_219	2008-04-08	12
+219	val_219	2008-04-08	12
+221	val_221	2008-04-08	12
+221	val_221	2008-04-08	12
+222	val_222	2008-04-08	12
+223	val_223	2008-04-08	12
+223	val_223	2008-04-08	12
+224	val_224	2008-04-08	12
+224	val_224	2008-04-08	12
+226	val_226	2008-04-08	12
+228	val_228	2008-04-08	12
+229	val_229	2008-04-08	12
+229	val_229	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+233	val_233	2008-04-08	12
+233	val_233	2008-04-08	12
+235	val_235	2008-04-08	12
+237	val_237	2008-04-08	12
+237	val_237	2008-04-08	12
+238	val_238	2008-04-08	12
+238	val_238	2008-04-08	12
+239	val_239	2008-04-08	12
+239	val_239	2008-04-08	12
+241	val_241	2008-04-08	12
+242	val_242	2008-04-08	12
+242	val_242	2008-04-08	12
+244	val_244	2008-04-08	12
+247	val_247	2008-04-08	12
+248	val_248	2008-04-08	12
+249	val_249	2008-04-08	12
+252	val_252	2008-04-08	12
+255	val_255	2008-04-08	12
+255	val_255	2008-04-08	12
+256	val_256	2008-04-08	12
+256	val_256	2008-04-08	12
+257	val_257	2008-04-08	12
+258	val_258	2008-04-08	12
+260	val_260	2008-04-08	12
+262	val_262	2008-04-08	12
+263	val_263	2008-04-08	12
+265	val_265	2008-04-08	12
+265	val_265	2008-04-08	12
+266	val_266	2008-04-08	12
+272	val_272	2008-04-08	12
+272	val_272	2008-04-08	12
+273	val_273	2008-04-08	12
+273	val_273	2008-04-08	12
+273	val_273	2008-04-08	12
+274	val_274	2008-04-08	12
+275	val_275	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+278	val_278	2008-04-08	12
+278	val_278	2008-04-08	12
+280	val_280	2008-04-08	12
+280	val_280	2008-04-08	12
+281	val_281	2008-04-08	12
+281	val_281	2008-04-08	12
+282	val_282	2008-04-08	12
+282	val_282	2008-04-08	12
+283	val_283	2008-04-08	12
+284	val_284	2008-04-08	12
+285	val_285	2008-04-08	12
+286	val_286	2008-04-08	12
+287	val_287	2008-04-08	12
+288	val_288	2008-04-08	12
+288	val_288	2008-04-08	12
+289	val_289	2008-04-08	12
+291	val_291	2008-04-08	12
+292	val_292	2008-04-08	12
+296	val_296	2008-04-08	12
+298	val_298	2008-04-08	12
+298	val_298	2008-04-08	12
+298	val_298	2008-04-08	12
+302	val_302	2008-04-08	12
+305	val_305	2008-04-08	12
+306	val_306	2008-04-08	12
+307	val_307	2008-04-08	12
+307	val_307	2008-04-08	12
+308	val_308	2008-04-08	12
+309	val_309	2008-04-08	12
+309	val_309	2008-04-08	12
+310	val_310	2008-04-08	12
+311	val_311	2008-04-08	12
+311	val_311	2008-04-08	12
+311	val_311	2008-04-08	12
+315	val_315	2008-04-08	12
+316	val_316	2008-04-08	12
+316	val_316	2008-04-08	12
+316	val_316	2008-04-08	12
+317	val_317	2008-04-08	12
+317	val_317	2008-04-08	12
+318	val_318	2008-04-08	12
+318	val_318	2008-04-08	12
+318	val_318	2008-04-08	12
+321	val_321	2008-04-08	12
+321	val_321	2008-04-08	12
+322	val_322	2008-04-08	12
+322	val_322	2008-04-08	12
+323	val_323	2008-04-08	12
+325	val_325	2008-04-08	12
+325	val_325	2008-04-08	12
+327	val_327	2008-04-08	12
+327	val_327	2008-04-08	12
+327	val_327	2008-04-08	12
+331	val_331	2008-04-08	12
+331	val_331	2008-04-08	12
+332	val_332	2008-04-08	12
+333	val_333	2008-04-08	12
+333	val_333	2008-04-08	12
+335	val_335	2008-04-08	12
+336	val_336	2008-04-08	12
+338	val_338	2008-04-08	12
+339	val_339	2008-04-08	12
+341	val_341	2008-04-08	12
+342	val_342	2008-04-08	12
+342	val_342	2008-04-08	12
+344	val_344	2008-04-08	12
+344	val_344	2008-04-08	12
+345	val_345	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+351	val_351	2008-04-08	12
+353	val_353	2008-04-08	12
+353	val_353	2008-04-08	12
+356	val_356	2008-04-08	12
+360	val_360	2008-04-08	12
+362	val_362	2008-04-08	12
+364	val_364	2008-04-08	12
+365	val_365	2008-04-08	12
+366	val_366	2008-04-08	12
+367	val_367	2008-04-08	12
+367	val_367	2008-04-08	12
+368	val_368	2008-04-08	12
+369	val_369	2008-04-08	12
+369	val_369	2008-04-08	12
+369	val_369	2008-04-08	12
+373	val_373	2008-04-08	12
+374	val_374	2008-04-08	12
+375	val_375	2008-04-08	12
+377	val_377	2008-04-08	12
+378	val_378	2008-04-08	12
+379	val_379	2008-04-08	12
+382	val_382	2008-04-08	12
+382	val_382	2008-04-08	12
+384	val_384	2008-04-08	12
+384	val_384	2008-04-08	12
+384	val_384	2008-04-08	12
+386	val_386	2008-04-08	12
+389	val_389	2008-04-08	12
+392	val_392	2008-04-08	12
+393	val_393	2008-04-08	12
+394	val_394	2008-04-08	12
+395	val_395	2008-04-08	12
+395	val_395	2008-04-08	12
+396	val_396	2008-04-08	12
+396	val_396	2008-04-08	12
+396	val_396	2008-04-08	12
+397	val_397	2008-04-08	12
+397	val_397	2008-04-08	12
+399	val_399	2008-04-08	12
+399	val_399	2008-04-08	12
+400	val_400	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+402	val_402	2008-04-08	12
+403	val_403	2008-04-08	12
+403	val_403	2008-04-08	12
+403	val_403	2008-04-08	12
+404	val_404	2008-04-08	12
+404	val_404	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+407	val_407	2008-04-08	12
+409	val_409	2008-04-08	12
+409	val_409	2008-04-08	12
+409	val_409	2008-04-08	12
+411	val_411	2008-04-08	12
+413	val_413	2008-04-08	12
+413	val_413	2008-04-08	12
+414	val_414	2008-04-08	12
+414	val_414	2008-04-08	12
+417	val_417	2008-04-08	12
+417	val_417	2008-04-08	12
+417	val_417	2008-04-08	12
+418	val_418	2008-04-08	12
+419	val_419	2008-04-08	12
+421	val_421	2008-04-08	12
+424	val_424	2008-04-08	12
+424	val_424	2008-04-08	12
+427	val_427	2008-04-08	12
+429	val_429	2008-04-08	12
+429	val_429	2008-04-08	12
+430	val_430	2008-04-08	12
+430	val_430	2008-04-08	12
+430	val_430	2008-04-08	12
+431	val_431	2008-04-08	12
+431	val_431	2008-04-08	12
+431	val_431	2008-04-08	12
+432	val_432	2008-04-08	12
+435	val_435	2008-04-08	12
+436	val_436	2008-04-08	12
+437	val_437	2008-04-08	12
+438	val_438	2008-04-08	12
+438	val_438	2008-04-08	12
+438	val_438	2008-04-08	12
+439	val_439	2008-04-08	12
+439	val_439	2008-04-08	12
+443	val_443	2008-04-08	12
+444	val_444	2008-04-08	12
+446	val_446	2008-04-08	12
+448	val_448	2008-04-08	12
+449	val_449	2008-04-08	12
+452	val_452	2008-04-08	12
+453	val_453	2008-04-08	12
+454	val_454	2008-04-08	12
+454	val_454	2008-04-08	12
+454	val_454	2008-04-08	12
+455	val_455	2008-04-08	12
+457	val_457	2008-04-08	12
+458	val_458	2008-04-08	12
+458	val_458	2008-04-08	12
+459	val_459	2008-04-08	12
+459	val_459	2008-04-08	12
+460	val_460	2008-04-08	12
+462	val_462	2008-04-08	12
+462	val_462	2008-04-08	12
+463	val_463	2008-04-08	12
+463	val_463	2008-04-08	12
+466	val_466	2008-04-08	12
+466	val_466	2008-04-08	12
+466	val_466	2008-04-08	12
+467	val_467	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+470	val_470	2008-04-08	12
+472	val_472	2008-04-08	12
+475	val_475	2008-04-08	12
+477	val_477	2008-04-08	12
+478	val_478	2008-04-08	12
+478	val_478	2008-04-08	12
+479	val_479	2008-04-08	12
+480	val_480	2008-04-08	12
+480	val_480	2008-04-08	12
+480	val_480	2008-04-08	12
+481	val_481	2008-04-08	12
+482	val_482	2008-04-08	12
+483	val_483	2008-04-08	12
+484	val_484	2008-04-08	12
+485	val_485	2008-04-08	12
+487	val_487	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+490	val_490	2008-04-08	12
+491	val_491	2008-04-08	12
+492	val_492	2008-04-08	12
+492	val_492	2008-04-08	12
+493	val_493	2008-04-08	12
+494	val_494	2008-04-08	12
+495	val_495	2008-04-08	12
+496	val_496	2008-04-08	12
+497	val_497	2008-04-08	12
+498	val_498	2008-04-08	12
+498	val_498	2008-04-08	12
+498	val_498	2008-04-08	12
+0	val_0	2008-04-08	existing_value
+0	val_0	2008-04-08	existing_value
+0	val_0	2008-04-08	existing_value
+2	val_2	2008-04-08	existing_value
+4	val_4	2008-04-08	existing_value
+5	val_5	2008-04-08	existing_value
+5	val_5	2008-04-08	existing_value
+5	val_5	2008-04-08	existing_value
+8	val_8	2008-04-08	existing_value
+9	val_9	2008-04-08	existing_value
+10	val_10	2008-04-08	existing_value
+11	val_11	2008-04-08	existing_value
+12	val_12	2008-04-08	existing_value
+12	val_12	2008-04-08	existing_value
+15	val_15	2008-04-08	existing_value
+15	val_15	2008-04-08	existing_value
+17	val_17	2008-04-08	existing_value
+18	val_18	2008-04-08	existing_value
+18	val_18	2008-04-08	existing_value
+19	val_19	2008-04-08	existing_value
+20	val_20	2008-04-08	existing_value
+24	val_24	2008-04-08	existing_value
+24	val_24	2008-04-08	existing_value
+26	val_26	2008-04-08	existing_value
+26	val_26	2008-04-08	existing_value
+27	val_27	2008-04-08	existing_value
+28	val_28	2008-04-08	existing_value
+30	val_30	2008-04-08	existing_value
+33	val_33	2008-04-08	existing_value
+34	val_34	2008-04-08	existing_value
+35	val_35	2008-04-08	existing_value
+35	val_35	2008-04-08	existing_value
+35	val_35	2008-04-08	existing_value
+37	val_37	2008-04-08	existing_value
+37	val_37	2008-04-08	existing_value
+41	val_41	2008-04-08	existing_value
+42	val_42	2008-04-08	existing_value
+42	val_42	2008-04-08	existing_value
+43	val_43	2008-04-08	existing_value
+44	val_44	2008-04-08	existing_value
+47	val_47	2008-04-08	existing_value
+51	val_51	2008-04-08	existing_value
+51	val_51	2008-04-08	existing_value
+53	val_53	2008-04-08	existing_value
+54	val_54	2008-04-08	existing_value
+57	val_57	2008-04-08	existing_value
+58	val_58	2008-04-08	existing_value
+58	val_58	2008-04-08	existing_value
+64	val_64	2008-04-08	existing_value
+65	val_65	2008-04-08	existing_value
+66	val_66	2008-04-08	existing_value
+67	val_67	2008-04-08	existing_value
+67	val_67	2008-04-08	existing_value
+69	val_69	2008-04-08	existing_value
+70	val_70	2008-04-08	existing_value
+70	val_70	2008-04-08	existing_value
+70	val_70	2008-04-08	existing_value
+72	val_72	2008-04-08	existing_value
+72	val_72	2008-04-08	existing_value
+74	val_74	2008-04-08	existing_value
+76	val_76	2008-04-08	existing_value
+76	val_76	2008-04-08	existing_value
+77	val_77	2008-04-08	existing_value
+78	val_78	2008-04-08	existing_value
+80	val_80	2008-04-08	existing_value
+82	val_82	2008-04-08	existing_value
+83	val_83	2008-04-08	existing_value
+83	val_83	2008-04-08	existing_value
+84	val_84	2008-04-08	existing_value
+84	val_84	2008-04-08	existing_value
+85	val_85	2008-04-08	existing_value
+86	val_86	2008-04-08	existing_value
+87	val_87	2008-04-08	existing_value
+90	val_90	2008-04-08	existing_value
+90	val_90	2008-04-08	existing_value
+90	val_90	2008-04-08	existing_value
+92	val_92	2008-04-08	existing_value
+95	val_95	2008-04-08	existing_value
+95	val_95	2008-04-08	existing_value
+96	val_96	2008-04-08	existing_value
+97	val_97	2008-04-08	existing_value
+97	val_97	2008-04-08	existing_value
+98	val_98	2008-04-08	existing_value
+98	val_98	2008-04-08	existing_value
+100	val_100	2008-04-08	existing_value
+100	val_100	2008-04-08	existing_value
+103	val_103	2008-04-08	existing_value
+103	val_103	2008-04-08	existing_value
+104	val_104	2008-04-08	existing_value
+104	val_104	2008-04-08	existing_value
+105	val_105	2008-04-08	existing_value
+111	val_111	2008-04-08	existing_value
+113	val_113	2008-04-08	existing_value
+113	val_113	2008-04-08	existing_value
+114	val_114	2008-04-08	existing_value
+116	val_116	2008-04-08	existing_value
+118	val_118	2008-04-08	existing_value
+118	val_118	2008-04-08	existing_value
+119	val_119	2008-04-08	existing_value
+119	val_119	2008-04-08	existing_value
+119	val_119	2008-04-08	existing_value
+120	val_120	2008-04-08	existing_value
+120	val_120	2008-04-08	existing_value
+125	val_125	2008-04-08	existing_value
+125	val_125	2008-04-08	existing_value
+126	val_126	2008-04-08	existing_value
+128	val_128	2008-04-08	existing_value
+128	val_128	2008-04-08	existing_value
+128	val_128	2008-04-08	existing_value
+129	val_129	2008-04-08	existing_value
+129	val_129	2008-04-08	existing_value
+131	val_131	2008-04-08	existing_value
+133	val_133	2008-04-08	existing_value
+134	val_134	2008-04-08	existing_value
+134	val_134	2008-04-08	existing_value
+136	val_136	2008-04-08	existing_value
+137	val_137	2008-04-08	existing_value
+137	val_137	2008-04-08	existing_value
+138	val_138	2008-04-08	existing_value
+138	val_138	2008-04-08	existing_value
+138	val_138	2008-04-08	existing_value
+138	val_138	2008-04-08	existing_value
+143	val_143	2008-04-08	existing_value
+145	val_145	2008-04-08	existing_value
+146	val_146	2008-04-08	existing_value
+146	val_146	2008-04-08	existing_value
+149	val_149	2008-04-08	existing_value
+149	val_149	2008-04-08	existing_value
+150	val_150	2008-04-08	existing_value
+152	val_152	2008-04-08	existing_value
+152	val_152	2008-04-08	existing_value
+153	val_153	2008-04-08	existing_value
+155	val_155	2008-04-08	existing_value
+156	val_156	2008-04-08	existing_value
+157	val_157	2008-04-08	existing_value
+158	val_158	2008-04-08	existing_value
+160	val_160	2008-04-08	existing_value
+162	val_162	2008-04-08	existing_value
+163	val_163	2008-04-08	existing_value
+164	val_164	2008-04-08	existing_value
+164	val_164	2008-04-08	existing_value
+165	val_165	2008-04-08	existing_value
+165	val_165	2008-04-08	existing_value
+166	val_166	2008-04-08	existing_value
+167	val_167	2008-04-08	existing_value
+167	val_167	2008-04-08	existing_value
+167	val_167	2008-04-08	existing_value
+168	val_168	2008-04-08	existing_value
+169	val_169	2008-04-08	existing_value
+169	val_169	2008-04-08	existing_value
+169	val_169	2008-04-08	existing_value
+169	val_169	2008-04-08	existing_value
+170	val_170	2008-04-08	existing_value
+172	val_172	2008-04-08	existing_value
+172	val_172	2008-04-08	existing_value
+174	val_174	2008-04-08	existing_value
+174	val_174	2008-04-08	existing_value
+175	val_175	2008-04-08	existing_value
+175	val_175	2008-04-08	existing_value
+176	val_176	2008-04-08	existing_value
+176	val_176	2008-04-08	existing_value
+177	val_177	2008-04-08	existing_value
+178	val_178	2008-04-08	existing_value
+179	val_179	2008-04-08	existing_value
+179	val_179	2008-04-08	existing_value
+180	val_180	2008-04-08	existing_value
+181	val_181	2008-04-08	existing_value
+183	val_183	2008-04-08	existing_value
+186	val_186	2008-04-08	existing_value
+187	val_187	2008-04-08	existing_value
+187	val_187	2008-04-08	existing_value
+187	val_187	2008-04-08	existing_value
+189	val_189	2008-04-08	existing_value
+190	val_190	2008-04-08	existing_value
+191	val_191	2008-04-08	existing_value
+191	val_191	2008-04-08	existing_value
+192	val_192	2008-04-08	existing_value
+193	val_193	2008-04-08	existing_value
+193	val_193	2008-04-08	existing_value
+193	val_193	2008-04-08	existing_value
+194	val_194	2008-04-08	existing_value
+195	val_195	2008-04-08	existing_value
+195	val_195	2008-04-08	existing_value
+196	val_196	2008-04-08	existing_value
+197	val_197	2008-04-08	existing_value
+197	val_197	2008-04-08	existing_value
+199	val_199	2008-04-08	existing_value
+199	val_199	2008-04-08	existing_value
+199	val_199	2008-04-08	existing_value
+200	val_200	2008-04-08	existing_value
+200	val_200	2008-04-08	existing_value
+201	val_201	2008-04-08	existing_value
+202	val_202	2008-04-08	existing_value
+203	val_203	2008-04-08	existing_value
+203	val_203	2008-04-08	existing_value
+205	val_205	2008-04-08	existing_value
+205	val_205	2008-04-08	existing_value
+207	val_207	2008-04-08	existing_value
+207	val_207	2008-04-08	existing_value
+208	val_208	2008-04-08	existing_value
+208	val_208	2008-04-08	existing_value
+208	val_208	2008-04-08	existing_value
+209	val_209	2008-04-08	existing_value
+209	val_209	2008-04-08	existing_value
+213	val_213	2008-04-08	existing_value
+213	val_213	2008-04-08	existing_value
+214	val_214	2008-04-08	existing_value
+216	val_216	2008-04-08	existing_value
+216	val_216	2008-04-08	existing_value
+217	val_217	2008-04-08	existing_value
+217	val_217	2008-04-08	existing_value
+218	val_218	2008-04-08	existing_value
+219	val_219	2008-04-08	existing_value
+219	val_219	2008-04-08	existing_value
+221	val_221	2008-04-08	existing_value
+221	val_221	2008-04-08	existing_value
+222	val_222	2008-04-08	existing_value
+223	val_223	2008-04-08	existing_value
+223	val_223	2008-04-08	existing_value
+224	val_224	2008-04-08	existing_value
+224	val_224	2008-04-08	existing_value
+226	val_226	2008-04-08	existing_value
+228	val_228	2008-04-08	existing_value
+229	val_229	2008-04-08	existing_value
+229	val_229	2008-04-08	existing_value
+230	val_230	2008-04-08	existing_value
+230	val_230	2008-04-08	existing_value
+230	val_230	2008-04-08	existing_value
+230	val_230	2008-04-08	existing_value
+230	val_230	2008-04-08	existing_value
+233	val_233	2008-04-08	existing_value
+233	val_233	2008-04-08	existing_value
+235	val_235	2008-04-08	existing_value
+237	val_237	2008-04-08	existing_value
+237	val_237	2008-04-08	existing_value
+238	val_238	2008-04-08	existing_value
+238	val_238	2008-04-08	existing_value
+239	val_239	2008-04-08	existing_value
+239	val_239	2008-04-08	existing_value
+241	val_241	2008-04-08	existing_value
+242	val_242	2008-04-08	existing_value
+242	val_242	2008-04-08	existing_value
+244	val_244	2008-04-08	existing_value
+247	val_247	2008-04-08	existing_value
+248	val_248	2008-04-08	existing_value
+249	val_249	2008-04-08	existing_value
+252	val_252	2008-04-08	existing_value
+255	val_255	2008-04-08	existing_value
+255	val_255	2008-04-08	existing_value
+256	val_256	2008-04-08	existing_value
+256	val_256	2008-04-08	existing_value
+257	val_257	2008-04-08	existing_value
+258	val_258	2008-04-08	existing_value
+260	val_260	2008-04-08	existing_value
+262	val_262	2008-04-08	existing_value
+263	val_263	2008-04-08	existing_value
+265	val_265	2008-04-08	existing_value
+265	val_265	2008-04-08	existing_value
+266	val_266	2008-04-08	existing_value
+272	val_272	2008-04-08	existing_value
+272	val_272	2008-04-08	existing_value
+273	val_273	2008-04-08	existing_value
+273	val_273	2008-04-08	existing_value
+273	val_273	2008-04-08	existing_value
+274	val_274	2008-04-08	existing_value
+275	val_275	2008-04-08	existing_value
+277	val_277	2008-04-08	existing_value
+277	val_277	2008-04-08	existing_value
+277	val_277	2008-04-08	existing_value
+277	val_277	2008-04-08	existing_value
+278	val_278	2008-04-08	existing_value
+278	val_278	2008-04-08	existing_value
+280	val_280	2008-04-08	existing_value
+280	val_280	2008-04-08	existing_value
+281	val_281	2008-04-08	existing_value
+281	val_281	2008-04-08	existing_value
+282	val_282	2008-04-08	existing_value
+282	val_282	2008-04-08	existing_value
+283	val_283	2008-04-08	existing_value
+284	val_284	2008-04-08	existing_value
+285	val_285	2008-04-08	existing_value
+286	val_286	2008-04-08	existing_value
+287	val_287	2008-04-08	existing_value
+288	val_288	2008-04-08	existing_value
+288	val_288	2008-04-08	existing_value
+289	val_289	2008-04-08	existing_value
+291	val_291	2008-04-08	existing_value
+292	val_292	2008-04-08	existing_value
+296	val_296	2008-04-08	existing_value
+298	val_298	2008-04-08	existing_value
+298	val_298	2008-04-08	existing_value
+298	val_298	2008-04-08	existing_value
+302	val_302	2008-04-08	existing_value
+305	val_305	2008-04-08	existing_value
+306	val_306	2008-04-08	existing_value
+307	val_307	2008-04-08	existing_value
+307	val_307	2008-04-08	existing_value
+308	val_308	2008-04-08	existing_value
+309	val_309	2008-04-08	existing_value
+309	val_309	2008-04-08	existing_value
+310	val_310	2008-04-08	existing_value
+311	val_311	2008-04-08	existing_value
+311	val_311	2008-04-08	existing_value
+311	val_311	2008-04-08	existing_value
+315	val_315	2008-04-08	existing_value
+316	val_316	2008-04-08	existing_value
+316	val_316	2008-04-08	existing_value
+316	val_316	2008-04-08	existing_value
+317	val_317	2008-04-08	existing_value
+317	val_317	2008-04-08	existing_value
+318	val_318	2008-04-08	existing_value
+318	val_318	2008-04-08	existing_value
+318	val_318	2008-04-08	existing_value
+321	val_321	2008-04-08	existing_value
+321	val_321	2008-04-08	existing_value
+322	val_322	2008-04-08	existing_value
+322	val_322	2008-04-08	existing_value
+323	val_323	2008-04-08	existing_value
+325	val_325	2008-04-08	existing_value
+325	val_325	2008-04-08	existing_value
+327	val_327	2008-04-08	existing_value
+327	val_327	2008-04-08	existing_value
+327	val_327	2008-04-08	existing_value
+331	val_331	2008-04-08	existing_value
+331	val_331	2008-04-08	existing_value
+332	val_332	2008-04-08	existing_value
+333	val_333	2008-04-08	existing_value
+333	val_333	2008-04-08	existing_value
+335	val_335	2008-04-08	existing_value
+336	val_336	2008-04-08	existing_value
+338	val_338	2008-04-08	existing_value
+339	val_339	2008-04-08	existing_value
+341	val_341	2008-04-08	existing_value
+342	val_342	2008-04-08	existing_value
+342	val_342	2008-04-08	existing_value
+344	val_344	2008-04-08	existing_value
+344	val_344	2008-04-08	existing_value
+345	val_345	2008-04-08	existing_value
+348	val_348	2008-04-08	existing_value
+348	val_348	2008-04-08	existing_value
+348	val_348	2008-04-08	existing_value
+348	val_348	2008-04-08	existing_value
+348	val_348	2008-04-08	existing_value
+351	val_351	2008-04-08	existing_value
+353	val_353	2008-04-08	existing_value
+353	val_353	2008-04-08	existing_value
+356	val_356	2008-04-08	existing_value
+360	val_360	2008-04-08	existing_value
+362	val_362	2008-04-08	existing_value
+364	val_364	2008-04-08	existing_value
+365	val_365	2008-04-08	existing_value
+366	val_366	2008-04-08	existing_value
+367	val_367	2008-04-08	existing_value
+367	val_367	2008-04-08	existing_value
+368	val_368	2008-04-08	existing_value
+369	val_369	2008-04-08	existing_value
+369	val_369	2008-04-08	existing_value
+369	val_369	2008-04-08	existing_value
+373	val_373	2008-04-08	existing_value
+374	val_374	2008-04-08	existing_value
+375	val_375	2008-04-08	existing_value
+377	val_377	2008-04-08	existing_value
+378	val_378	2008-04-08	existing_value
+379	val_379	2008-04-08	existing_value
+382	val_382	2008-04-08	existing_value
+382	val_382	2008-04-08	existing_value
+384	val_384	2008-04-08	existing_value
+384	val_384	2008-04-08	existing_value
+384	val_384	2008-04-08	existing_value
+386	val_386	2008-04-08	existing_value
+389	val_389	2008-04-08	existing_value
+392	val_392	2008-04-08	existing_value
+393	val_393	2008-04-08	existing_value
+394	val_394	2008-04-08	existing_value
+395	val_395	2008-04-08	existing_value
+395	val_395	2008-04-08	existing_value
+396	val_396	2008-04-08	existing_value
+396	val_396	2008-04-08	existing_value
+396	val_396	2008-04-08	existing_value
+397	val_397	2008-04-08	existing_value
+397	val_397	2008-04-08	existing_value
+399	val_399	2008-04-08	existing_value
+399	val_399	2008-04-08	existing_value
+400	val_400	2008-04-08	existing_value
+401	val_401	2008-04-08	existing_value
+401	val_401	2008-04-08	existing_value
+401	val_401	2008-04-08	existing_value
+401	val_401	2008-04-08	existing_value
+401	val_401	2008-04-08	existing_value
+402	val_402	2008-04-08	existing_value
+403	val_403	2008-04-08	existing_value
+403	val_403	2008-04-08	existing_value
+403	val_403	2008-04-08	existing_value
+404	val_404	2008-04-08	existing_value
+404	val_404	2008-04-08	existing_value
+406	val_406	2008-04-08	existing_value
+406	val_406	2008-04-08	existing_value
+406	val_406	2008-04-08	existing_value
+406	val_406	2008-04-08	existing_value
+407	val_407	2008-04-08	existing_value
+409	val_409	2008-04-08	existing_value
+409	val_409	2008-04-08	existing_value
+409	val_409	2008-04-08	existing_value
+411	val_411	2008-04-08	existing_value
+413	val_413	2008-04-08	existing_value
+413	val_413	2008-04-08	existing_value
+414	val_414	2008-04-08	existing_value
+414	val_414	2008-04-08	existing_value
+417	val_417	2008-04-08	existing_value
+417	val_417	2008-04-08	existing_value
+417	val_417	2008-04-08	existing_value
+418	val_418	2008-04-08	existing_value
+419	val_419	2008-04-08	existing_value
+421	val_421	2008-04-08	existing_value
+424	val_424	2008-04-08	existing_value
+424	val_424	2008-04-08	existing_value
+427	val_427	2008-04-08	existing_value
+429	val_429	2008-04-08	existing_value
+429	val_429	2008-04-08	existing_value
+430	val_430	2008-04-08	existing_value
+430	val_430	2008-04-08	existing_value
+430	val_430	2008-04-08	existing_value
+431	val_431	2008-04-08	existing_value
+431	val_431	2008-04-08	existing_value
+431	val_431	2008-04-08	existing_value
+432	val_432	2008-04-08	existing_value
+435	val_435	2008-04-08	existing_value
+436	val_436	2008-04-08	existing_value
+437	val_437	2008-04-08	existing_value
+438	val_438	2008-04-08	existing_value
+438	val_438	2008-04-08	existing_value
+438	val_438	2008-04-08	existing_value
+439	val_439	2008-04-08	existing_value
+439	val_439	2008-04-08	existing_value
+443	val_443	2008-04-08	existing_value
+444	val_444	2008-04-08	existing_value
+446	val_446	2008-04-08	existing_value
+448	val_448	2008-04-08	existing_value
+449	val_449	2008-04-08	existing_value
+452	val_452	2008-04-08	existing_value
+453	val_453	2008-04-08	existing_value
+454	val_454	2008-04-08	existing_value
+454	val_454	2008-04-08	existing_value
+454	val_454	2008-04-08	existing_value
+455	val_455	2008-04-08	existing_value
+457	val_457	2008-04-08	existing_value
+458	val_458	2008-04-08	existing_value
+458	val_458	2008-04-08	existing_value
+459	val_459	2008-04-08	existing_value
+459	val_459	2008-04-08	existing_value
+460	val_460	2008-04-08	existing_value
+462	val_462	2008-04-08	existing_value
+462	val_462	2008-04-08	existing_value
+463	val_463	2008-04-08	existing_value
+463	val_463	2008-04-08	existing_value
+466	val_466	2008-04-08	existing_value
+466	val_466	2008-04-08	existing_value
+466	val_466	2008-04-08	existing_value
+467	val_467	2008-04-08	existing_value
+468	val_468	2008-04-08	existing_value
+468	val_468	2008-04-08	existing_value
+468	val_468	2008-04-08	existing_value
+468	val_468	2008-04-08	existing_value
+469	val_469	2008-04-08	existing_value
+469	val_469	2008-04-08	existing_value
+469	val_469	2008-04-08	existing_value
+469	val_469	2008-04-08	existing_value
+469	val_469	2008-04-08	existing_value
+470	val_470	2008-04-08	existing_value
+472	val_472	2008-04-08	existing_value
+475	val_475	2008-04-08	existing_value
+477	val_477	2008-04-08	existing_value
+478	val_478	2008-04-08	existing_value
+478	val_478	2008-04-08	existing_value
+479	val_479	2008-04-08	existing_value
+480	val_480	2008-04-08	existing_value
+480	val_480	2008-04-08	existing_value
+480	val_480	2008-04-08	existing_value
+481	val_481	2008-04-08	existing_value
+482	val_482	2008-04-08	existing_value
+483	val_483	2008-04-08	existing_value
+484	val_484	2008-04-08	existing_value
+485	val_485	2008-04-08	existing_value
+487	val_487	2008-04-08	existing_value
+489	val_489	2008-04-08	existing_value
+489	val_489	2008-04-08	existing_value
+489	val_489	2008-04-08	existing_value
+489	val_489	2008-04-08	existing_value
+490	val_490	2008-04-08	existing_value
+491	val_491	2008-04-08	existing_value
+492	val_492	2008-04-08	existing_value
+492	val_492	2008-04-08	existing_value
+493	val_493	2008-04-08	existing_value
+494	val_494	2008-04-08	existing_value
+495	val_495	2008-04-08	existing_value
+496	val_496	2008-04-08	existing_value
+497	val_497	2008-04-08	existing_value
+498	val_498	2008-04-08	existing_value
+498	val_498	2008-04-08	existing_value
+498	val_498	2008-04-08	existing_value
+0	val_0	2008-04-09	11
+0	val_0	2008-04-09	11
+0	val_0	2008-04-09	11
+2	val_2	2008-04-09	11
+4	val_4	2008-04-09	11
+5	val_5	2008-04-09	11
+5	val_5	2008-04-09	11
+5	val_5	2008-04-09	11
+8	val_8	2008-04-09	11
+9	val_9	2008-04-09	11
+10	val_10	2008-04-09	11
+11	val_11	2008-04-09	11
+12	val_12	2008-04-09	11
+12	val_12	2008-04-09	11
+15	val_15	2008-04-09	11
+15	val_15	2008-04-09	11
+17	val_17	2008-04-09	11
+18	val_18	2008-04-09	11
+18	val_18	2008-04-09	11
+19	val_19	2008-04-09	11
+20	val_20	2008-04-09	11
+24	val_24	2008-04-09	11
+24	val_24	2008-04-09	11
+26	val_26	2008-04-09	11
+26	val_26	2008-04-09	11
+27	val_27	2008-04-09	11
+28	val_28	2008-04-09	11
+30	val_30	2008-04-09	11
+33	val_33	2008-04-09	11
+34	val_34	2008-04-09	11
+35	val_35	2008-04-09	11
+35	val_35	2008-04-09	11
+35	val_35	2008-04-09	11
+37	val_37	2008-04-09	11
+37	val_37	2008-04-09	11
+41	val_41	2008-04-09	11
+42	val_42	2008-04-09	11
+42	val_42	2008-04-09	11
+43	val_43	2008-04-09	11
+44	val_44	2008-04-09	11
+47	val_47	2008-04-09	11
+51	val_51	2008-04-09	11
+51	val_51	2008-04-09	11
+53	val_53	2008-04-09	11
+54	val_54	2008-04-09	11
+57	val_57	2008-04-09	11
+58	val_58	2008-04-09	11
+58	val_58	2008-04-09	11
+64	val_64	2008-04-09	11
+65	val_65	2008-04-09	11
+66	val_66	2008-04-09	11
+67	val_67	2008-04-09	11
+67	val_67	2008-04-09	11
+69	val_69	2008-04-09	11
+70	val_70	2008-04-09	11
+70	val_70	2008-04-09	11
+70	val_70	2008-04-09	11
+72	val_72	2008-04-09	11
+72	val_72	2008-04-09	11
+74	val_74	2008-04-09	11
+76	val_76	2008-04-09	11
+76	val_76	2008-04-09	11
+77	val_77	2008-04-09	11
+78	val_78	2008-04-09	11
+80	val_80	2008-04-09	11
+82	val_82	2008-04-09	11
+83	val_83	2008-04-09	11
+83	val_83	2008-04-09	11
+84	val_84	2008-04-09	11
+84	val_84	2008-04-09	11
+85	val_85	2008-04-09	11
+86	val_86	2008-04-09	11
+87	val_87	2008-04-09	11
+90	val_90	2008-04-09	11
+90	val_90	2008-04-09	11
+90	val_90	2008-04-09	11
+92	val_92	2008-04-09	11
+95	val_95	2008-04-09	11
+95	val_95	2008-04-09	11
+96	val_96	2008-04-09	11
+97	val_97	2008-04-09	11
+97	val_97	2008-04-09	11
+98	val_98	2008-04-09	11
+98	val_98	2008-04-09	11
+100	val_100	2008-04-09	11
+100	val_100	2008-04-09	11
+103	val_103	2008-04-09	11
+103	val_103	2008-04-09	11
+104	val_104	2008-04-09	11
+104	val_104	2008-04-09	11
+105	val_105	2008-04-09	11
+111	val_111	2008-04-09	11
+113	val_113	2008-04-09	11
+113	val_113	2008-04-09	11
+114	val_114	2008-04-09	11
+116	val_116	2008-04-09	11
+118	val_118	2008-04-09	11
+118	val_118	2008-04-09	11
+119	val_119	2008-04-09	11
+119	val_119	2008-04-09	11
+119	val_119	2008-04-09	11
+120	val_120	2008-04-09	11
+120	val_120	2008-04-09	11
+125	val_125	2008-04-09	11
+125	val_125	2008-04-09	11
+126	val_126	2008-04-09	11
+128	val_128	2008-04-09	11
+128	val_128	2008-04-09	11
+128	val_128	2008-04-09	11
+129	val_129	2008-04-09	11
+129	val_129	2008-04-09	11
+131	val_131	2008-04-09	11
+133	val_133	2008-04-09	11
+134	val_134	2008-04-09	11
+134	val_134	2008-04-09	11
+136	val_136	2008-04-09	11
+137	val_137	2008-04-09	11
+137	val_137	2008-04-09	11
+138	val_138	2008-04-09	11
+138	val_138	2008-04-09	11
+138	val_138	2008-04-09	11
+138	val_138	2008-04-09	11
+143	val_143	2008-04-09	11
+145	val_145	2008-04-09	11
+146	val_146	2008-04-09	11
+146	val_146	2008-04-09	11
+149	val_149	2008-04-09	11
+149	val_149	2008-04-09	11
+150	val_150	2008-04-09	11
+152	val_152	2008-04-09	11
+152	val_152	2008-04-09	11
+153	val_153	2008-04-09	11
+155	val_155	2008-04-09	11
+156	val_156	2008-04-09	11
+157	val_157	2008-04-09	11
+158	val_158	2008-04-09	11
+160	val_160	2008-04-09	11
+162	val_162	2008-04-09	11
+163	val_163	2008-04-09	11
+164	val_164	2008-04-09	11
+164	val_164	2008-04-09	11
+165	val_165	2008-04-09	11
+165	val_165	2008-04-09	11
+166	val_166	2008-04-09	11
+167	val_167	2008-04-09	11
+167	val_167	2008-04-09	11
+167	val_167	2008-04-09	11
+168	val_168	2008-04-09	11
+169	val_169	2008-04-09	11
+169	val_169	2008-04-09	11
+169	val_169	2008-04-09	11
+169	val_169	2008-04-09	11
+170	val_170	2008-04-09	11
+172	val_172	2008-04-09	11
+172	val_172	2008-04-09	11
+174	val_174	2008-04-09	11
+174	val_174	2008-04-09	11
+175	val_175	2008-04-09	11
+175	val_175	2008-04-09	11
+176	val_176	2008-04-09	11
+176	val_176	2008-04-09	11
+177	val_177	2008-04-09	11
+178	val_178	2008-04-09	11
+179	val_179	2008-04-09	11
+179	val_179	2008-04-09	11
+180	val_180	2008-04-09	11
+181	val_181	2008-04-09	11
+183	val_183	2008-04-09	11
+186	val_186	2008-04-09	11
+187	val_187	2008-04-09	11
+187	val_187	2008-04-09	11
+187	val_187	2008-04-09	11
+189	val_189	2008-04-09	11
+190	val_190	2008-04-09	11
+191	val_191	2008-04-09	11
+191	val_191	2008-04-09	11
+192	val_192	2008-04-09	11
+193	val_193	2008-04-09	11
+193	val_193	2008-04-09	11
+193	val_193	2008-04-09	11
+194	val_194	2008-04-09	11
+195	val_195	2008-04-09	11
+195	val_195	2008-04-09	11
+196	val_196	2008-04-09	11
+197	val_197	2008-04-09	11
+197	val_197	2008-04-09	11
+199	val_199	2008-04-09	11
+199	val_199	2008-04-09	11
+199	val_199	2008-04-09	11
+200	val_200	2008-04-09	11
+200	val_200	2008-04-09	11
+201	val_201	2008-04-09	11
+202	val_202	2008-04-09	11
+203	val_203	2008-04-09	11
+203	val_203	2008-04-09	11
+205	val_205	2008-04-09	11
+205	val_205	2008-04-09	11
+207	val_207	2008-04-09	11
+207	val_207	2008-04-09	11
+208	val_208	2008-04-09	11
+208	val_208	2008-04-09	11
+208	val_208	2008-04-09	11
+209	val_209	2008-04-09	11
+209	val_209	2008-04-09	11
+213	val_213	2008-04-09	11
+213	val_213	2008-04-09	11
+214	val_214	2008-04-09	11
+216	val_216	2008-04-09	11
+216	val_216	2008-04-09	11
+217	val_217	2008-04-09	11
+217	val_217	2008-04-09	11
+218	val_218	2008-04-09	11
+219	val_219	2008-04-09	11
+219	val_219	2008-04-09	11
+221	val_221	2008-04-09	11
+221	val_221	2008-04-09	11
+222	val_222	2008-04-09	11
+223	val_223	2008-04-09	11
+223	val_223	2008-04-09	11
+224	val_224	2008-04-09	11
+224	val_224	2008-04-09	11
+226	val_226	2008-04-09	11
+228	val_228	2008-04-09	11
+229	val_229	2008-04-09	11
+229	val_229	2008-04-09	11
+230	val_230	2008-04-09	11
+230	val_230	2008-04-09	11
+230	val_230	2008-04-09	11
+230	val_230	2008-04-09	11
+230	val_230	2008-04-09	11
+233	val_233	2008-04-09	11
+233	val_233	2008-04-09	11
+235	val_235	2008-04-09	11
+237	val_237	2008-04-09	11
+237	val_237	2008-04-09	11
+238	val_238	2008-04-09	11
+238	val_238	2008-04-09	11
+239	val_239	2008-04-09	11
+239	val_239	2008-04-09	11
+241	val_241	2008-04-09	11
+242	val_242	2008-04-09	11
+242	val_242	2008-04-09	11
+244	val_244	2008-04-09	11
+247	val_247	2008-04-09	11
+248	val_248	2008-04-09	11
+249	val_249	2008-04-09	11
+252	val_252	2008-04-09	11
+255	val_255	2008-04-09	11
+255	val_255	2008-04-09	11
+256	val_256	2008-04-09	11
+256	val_256	2008-04-09	11
+257	val_257	2008-04-09	11
+258	val_258	2008-04-09	11
+260	val_260	2008-04-09	11
+262	val_262	2008-04-09	11
+263	val_263	2008-04-09	11
+265	val_265	2008-04-09	11
+265	val_265	2008-04-09	11
+266	val_266	2008-04-09	11
+272	val_272	2008-04-09	11
+272	val_272	2008-04-09	11
+273	val_273	2008-04-09	11
+273	val_273	2008-04-09	11
+273	val_273	2008-04-09	11
+274	val_274	2008-04-09	11
+275	val_275	2008-04-09	11
+277	val_277	2008-04-09	11
+277	val_277	2008-04-09	11
+277	val_277	2008-04-09	11
+277	val_277	2008-04-09	11
+278	val_278	2008-04-09	11
+278	val_278	2008-04-09	11
+280	val_280	2008-04-09	11
+280	val_280	2008-04-09	11
+281	val_281	2008-04-09	11
+281	val_281	2008-04-09	11
+282	val_282	2008-04-09	11
+282	val_282	2008-04-09	11
+283	val_283	2008-04-09	11
+284	val_284	2008-04-09	11
+285	val_285	2008-04-09	11
+286	val_286	2008-04-09	11
+287	val_287	2008-04-09	11
+288	val_288	2008-04-09	11
+288	val_288	2008-04-09	11
+289	val_289	2008-04-09	11
+291	val_291	2008-04-09	11
+292	val_292	2008-04-09	11
+296	val_296	2008-04-09	11
+298	val_298	2008-04-09	11
+298	val_298	2008-04-09	11
+298	val_298	2008-04-09	11
+302	val_302	2008-04-09	11
+305	val_305	2008-04-09	11
+306	val_306	2008-04-09	11
+307	val_307	2008-04-09	11
+307	val_307	2008-04-09	11
+308	val_308	2008-04-09	11
+309	val_309	2008-04-09	11
+309	val_309	2008-04-09	11
+310	val_310	2008-04-09	11
+311	val_311	2008-04-09	11
+311	val_311	2008-04-09	11
+311	val_311	2008-04-09	11
+315	val_315	2008-04-09	11
+316	val_316	2008-04-09	11
+316	val_316	2008-04-09	11
+316	val_316	2008-04-09	11
+317	val_317	2008-04-09	11
+317	val_317	2008-04-09	11
+318	val_318	2008-04-09	11
+318	val_318	2008-04-09	11
+318	val_318	2008-04-09	11
+321	val_321	2008-04-09	11
+321	val_321	2008-04-09	11
+322	val_322	2008-04-09	11
+322	val_322	2008-04-09	11
+323	val_323	2008-04-09	11
+325	val_325	2008-04-09	11
+325	val_325	2008-04-09	11
+327	val_327	2008-04-09	11
+327	val_327	2008-04-09	11
+327	val_327	2008-04-09	11
+331	val_331	2008-04-09	11
+331	val_331	2008-04-09	11
+332	val_332	2008-04-09	11
+333	val_333	2008-04-09	11
+333	val_333	2008-04-09	11
+335	val_335	2008-04-09	11
+336	val_336	2008-04-09	11
+338	val_338	2008-04-09	11
+339	val_339	2008-04-09	11
+341	val_341	2008-04-09	11
+342	val_342	2008-04-09	11
+342	val_342	2008-04-09	11
+344	val_344	2008-04-09	11
+344	val_344	2008-04-09	11
+345	val_345	2008-04-09	11
+348	val_348	2008-04-09	11
+348	val_348	2008-04-09	11
+348	val_348	2008-04-09	11
+348	val_348	2008-04-09	11
+348	val_348	2008-04-09	11
+351	val_351	2008-04-09	11
+353	val_353	2008-04-09	11
+353	val_353	2008-04-09	11
+356	val_356	2008-04-09	11
+360	val_360	2008-04-09	11
+362	val_362	2008-04-09	11
+364	val_364	2008-04-09	11
+365	val_365	2008-04-09	11
+366	val_366	2008-04-09	11
+367	val_367	2008-04-09	11
+367	val_367	2008-04-09	11
+368	val_368	2008-04-09	11
+369	val_369	2008-04-09	11
+369	val_369	2008-04-09	11
+369	val_369	2008-04-09	11
+373	val_373	2008-04-09	11
+374	val_374	2008-04-09	11
+375	val_375	2008-04-09	11
+377	val_377	2008-04-09	11
+378	val_378	2008-04-09	11
+379	val_379	2008-04-09	11
+382	val_382	2008-04-09	11
+382	val_382	2008-04-09	11
+384	val_384	2008-04-09	11
+384	val_384	2008-04-09	11
+384	val_384	2008-04-09	11
+386	val_386	2008-04-09	11
+389	val_389	2008-04-09	11
+392	val_392	2008-04-09	11
+393	val_393	2008-04-09	11
+394	val_394	2008-04-09	11
+395	val_395	2008-04-09	11
+395	val_395	2008-04-09	11
+396	val_396	2008-04-09	11
+396	val_396	2008-04-09	11
+396	val_396	2008-04-09	11
+397	val_397	2008-04-09	11
+397	val_397	2008-04-09	11
+399	val_399	2008-04-09	11
+399	val_399	2008-04-09	11
+400	val_400	2008-04-09	11
+401	val_401	2008-04-09	11
+401	val_401	2008-04-09	11
+401	val_401	2008-04-09	11
+401	val_401	2008-04-09	11
+401	val_401	2008-04-09	11
+402	val_402	2008-04-09	11
+403	val_403	2008-04-09	11
+403	val_403	2008-04-09	11
+403	val_403	2008-04-09	11
+404	val_404	2008-04-09	11
+404	val_404	2008-04-09	11
+406	val_406	2008-04-09	11
+406	val_406	2008-04-09	11
+406	val_406	2008-04-09	11
+406	val_406	2008-04-09	11
+407	val_407	2008-04-09	11
+409	val_409	2008-04-09	11
+409	val_409	2008-04-09	11
+409	val_409	2008-04-09	11
+411	val_411	2008-04-09	11
+413	val_413	2008-04-09	11
+413	val_413	2008-04-09	11
+414	val_414	2008-04-09	11
+414	val_414	2008-04-09	11
+417	val_417	2008-04-09	11
+417	val_417	2008-04-09	11
+417	val_417	2008-04-09	11
+418	val_418	2008-04-09	11
+419	val_419	2008-04-09	11
+421	val_421	2008-04-09	11
+424	val_424	2008-04-09	11
+424	val_424	2008-04-09	11
+427	val_427	2008-04-09	11
+429	val_429	2008-04-09	11
+429	val_429	2008-04-09	11
+430	val_430	2008-04-09	11
+430	val_430	2008-04-09	11
+430	val_430	2008-04-09	11
+431	val_431	2008-04-09	11
+431	val_431	2008-04-09	11
+431	val_431	2008-04-09	11
+432	val_432	2008-04-09	11
+435	val_435	2008-04-09	11
+436	val_436	2008-04-09	11
+437	val_437	2008-04-09	11
+438	val_438	2008-04-09	11
+438	val_438	2008-04-09	11
+438	val_438	2008-04-09	11
+439	val_439	2008-04-09	11
+439	val_439	2008-04-09	11
+443	val_443	2008-04-09	11
+444	val_444	2008-04-09	11
+446	val_446	2008-04-09	11
+448	val_448	2008-04-09	11
+449	val_449	2008-04-09	11
+452	val_452	2008-04-09	11
+453	val_453	2008-04-09	11
+454	val_454	2008-04-09	11
+454	val_454	2008-04-09	11
+454	val_454	2008-04-09	11
+455	val_455	2008-04-09	11
+457	val_457	2008-04-09	11
+458	val_458	2008-04-09	11
+458	val_458	2008-04-09	11
+459	val_459	2008-04-09	11
+459	val_459	2008-04-09	11
+460	val_460	2008-04-09	11
+462	val_462	2008-04-09	11
+462	val_462	2008-04-09	11
+463	val_463	2008-04-09	11
+463	val_463	2008-04-09	11
+466	val_466	2008-04-09	11
+466	val_466	2008-04-09	11
+466	val_466	2008-04-09	11
+467	val_467	2008-04-09	11
+468	val_468	2008-04-09	11
+468	val_468	2008-04-09	11
+468	val_468	2008-04-09	11
+468	val_468	2008-04-09	11
+469	val_469	2008-04-09	11
+469	val_469	2008-04-09	11
+469	val_469	2008-04-09	11
+469	val_469	2008-04-09	11
+469	val_469	2008-04-09	11
+470	val_470	2008-04-09	11
+472	val_472	2008-04-09	11
+475	val_475	2008-04-09	11
+477	val_477	2008-04-09	11
+478	val_478	2008-04-09	11
+478	val_478	2008-04-09	11
+479	val_479	2008-04-09	11
+480	val_480	2008-04-09	11
+480	val_480	2008-04-09	11
+480	val_480	2008-04-09	11
+481	val_481	2008-04-09	11
+482	val_482	2008-04-09	11
+483	val_483	2008-04-09	11
+484	val_484	2008-04-09	11
+485	val_485	2008-04-09	11
+487	val_487	2008-04-09	11
+489	val_489	2008-04-09	11
+489	val_489	2008-04-09	11
+489	val_489	2008-04-09	11
+489	val_489	2008-04-09	11
+490	val_490	2008-04-09	11
+491	val_491	2008-04-09	11
+492	val_492	2008-04-09	11
+492	val_492	2008-04-09	11
+493	val_493	2008-04-09	11
+494	val_494	2008-04-09	11
+495	val_495	2008-04-09	11
+496	val_496	2008-04-09	11
+497	val_497	2008-04-09	11
+498	val_498	2008-04-09	11
+498	val_498	2008-04-09	11
+498	val_498	2008-04-09	11
+0	val_0	2008-04-09	12
+0	val_0	2008-04-09	12
+0	val_0	2008-04-09	12
+2	val_2	2008-04-09	12
+4	val_4	2008-04-09	12
+5	val_5	2008-04-09	12
+5	val_5	2008-04-09	12
+5	val_5	2008-04-09	12
+8	val_8	2008-04-09	12
+9	val_9	2008-04-09	12
+10	val_10	2008-04-09	12
+11	val_11	2008-04-09	12
+12	val_12	2008-04-09	12
+12	val_12	2008-04-09	12
+15	val_15	2008-04-09	12
+15	val_15	2008-04-09	12
+17	val_17	2008-04-09	12
+18	val_18	2008-04-09	12
+18	val_18	2008-04-09	12
+19	val_19	2008-04-09	12
+20	val_20	2008-04-09	12
+24	val_24	2008-04-09	12
+24	val_24	2008-04-09	12
+26	val_26	2008-04-09	12
+26	val_26	2008-04-09	12
+27	val_27	2008-04-09	12
+28	val_28	2008-04-09	12
+30	val_30	2008-04-09	12
+33	val_33	2008-04-09	12
+34	val_34	2008-04-09	12
+35	val_35	2008-04-09	12
+35	val_35	2008-04-09	12
+35	val_35	2008-04-09	12
+37	val_37	2008-04-09	12
+37	val_37	2008-04-09	12
+41	val_41	2008-04-09	12
+42	val_42	2008-04-09	12
+42	val_42	2008-04-09	12
+43	val_43	2008-04-09	12
+44	val_44	2008-04-09	12
+47	val_47	2008-04-09	12
+51	val_51	2008-04-09	12
+51	val_51	2008-04-09	12
+53	val_53	2008-04-09	12
+54	val_54	2008-04-09	12
+57	val_57	2008-04-09	12
+58	val_58	2008-04-09	12
+58	val_58	2008-04-09	12
+64	val_64	2008-04-09	12
+65	val_65	2008-04-09	12
+66	val_66	2008-04-09	12
+67	val_67	2008-04-09	12
+67	val_67	2008-04-09	12
+69	val_69	2008-04-09	12
+70	val_70	2008-04-09	12
+70	val_70	2008-04-09	12
+70	val_70	2008-04-09	12
+72	val_72	2008-04-09	12
+72	val_72	2008-04-09	12
+74	val_74	2008-04-09	12
+76	val_76	2008-04-09	12
+76	val_76	2008-04-09	12
+77	val_77	2008-04-09	12
+78	val_78	2008-04-09	12
+80	val_80	2008-04-09	12
+82	val_82	2008-04-09	12
+83	val_83	2008-04-09	12
+83	val_83	2008-04-09	12
+84	val_84	2008-04-09	12
+84	val_84	2008-04-09	12
+85	val_85	2008-04-09	12
+86	val_86	2008-04-09	12
+87	val_87	2008-04-09	12
+90	val_90	2008-04-09	12
+90	val_90	2008-04-09	12
+90	val_90	2008-04-09	12
+92	val_92	2008-04-09	12
+95	val_95	2008-04-09	12
+95	val_95	2008-04-09	12
+96	val_96	2008-04-09	12
+97	val_97	2008-04-09	12
+97	val_97	2008-04-09	12
+98	val_98	2008-04-09	12
+98	val_98	2008-04-09	12
+100	val_100	2008-04-09	12
+100	val_100	2008-04-09	12
+103	val_103	2008-04-09	12
+103	val_103	2008-04-09	12
+104	val_104	2008-04-09	12
+104	val_104	2008-04-09	12
+105	val_105	2008-04-09	12
+111	val_111	2008-04-09	12
+113	val_113	2008-04-09	12
+113	val_113	2008-04-09	12
+114	val_114	2008-04-09	12
+116	val_116	2008-04-09	12
+118	val_118	2008-04-09	12
+118	val_118	2008-04-09	12
+119	val_119	2008-04-09	12
+119	val_119	2008-04-09	12
+119	val_119	2008-04-09	12
+120	val_120	2008-04-09	12
+120	val_120	2008-04-09	12
+125	val_125	2008-04-09	12
+125	val_125	2008-04-09	12
+126	val_126	2008-04-09	12
+128	val_128	2008-04-09	12
+128	val_128	2008-04-09	12
+128	val_128	2008-04-09	12
+129	val_129	2008-04-09	12
+129	val_129	2008-04-09	12
+131	val_131	2008-04-09	12
+133	val_133	2008-04-09	12
+134	val_134	2008-04-09	12
+134	val_134	2008-04-09	12
+136	val_136	2008-04-09	12
+137	val_137	2008-04-09	12
+137	val_137	2008-04-09	12
+138	val_138	2008-04-09	12
+138	val_138	2008-04-09	12
+138	val_138	2008-04-09	12
+138	val_138	2008-04-09	12
+143	val_143	2008-04-09	12
+145	val_145	2008-04-09	12
+146	val_146	2008-04-09	12
+146	val_146	2008-04-09	12
+149	val_149	2008-04-09	12
+149	val_149	2008-04-09	12
+150	val_150	2008-04-09	12
+152	val_152	2008-04-09	12
+152	val_152	2008-04-09	12
+153	val_153	2008-04-09	12
+155	val_155	2008-04-09	12
+156	val_156	2008-04-09	12
+157	val_157	2008-04-09	12
+158	val_158	2008-04-09	12
+160	val_160	2008-04-09	12
+162	val_162	2008-04-09	12
+163	val_163	2008-04-09	12
+164	val_164	2008-04-09	12
+164	val_164	2008-04-09	12
+165	val_165	2008-04-09	12
+165	val_165	2008-04-09	12
+166	val_166	2008-04-09	12
+167	val_167	2008-04-09	12
+167	val_167	2008-04-09	12
+167	val_167	2008-04-09	12
+168	val_168	2008-04-09	12
+169	val_169	2008-04-09	12
+169	val_169	2008-04-09	12
+169	val_169	2008-04-09	12
+169	val_169	2008-04-09	12
+170	val_170	2008-04-09	12
+172	val_172	2008-04-09	12
+172	val_172	2008-04-09	12
+174	val_174	2008-04-09	12
+174	val_174	2008-04-09	12
+175	val_175	2008-04-09	12
+175	val_175	2008-04-09	12
+176	val_176	2008-04-09	12
+176	val_176	2008-04-09	12
+177	val_177	2008-04-09	12
+178	val_178	2008-04-09	12
+179	val_179	2008-04-09	12
+179	val_179	2008-04-09	12
+180	val_180	2008-04-09	12
+181	val_181	2008-04-09	12
+183	val_183	2008-04-09	12
+186	val_186	2008-04-09	12
+187	val_187	2008-04-09	12
+187	val_187	2008-04-09	12
+187	val_187	2008-04-09	12
+189	val_189	2008-04-09	12
+190	val_190	2008-04-09	12
+191	val_191	2008-04-09	12
+191	val_191	2008-04-09	12
+192	val_192	2008-04-09	12
+193	val_193	2008-04-09	12
+193	val_193	2008-04-09	12
+193	val_193	2008-04-09	12
+194	val_194	2008-04-09	12
+195	val_195	2008-04-09	12
+195	val_195	2008-04-09	12
+196	val_196	2008-04-09	12
+197	val_197	2008-04-09	12
+197	val_197	2008-04-09	12
+199	val_199	2008-04-09	12
+199	val_199	2008-04-09	12
+199	val_199	2008-04-09	12
+200	val_200	2008-04-09	12
+200	val_200	2008-04-09	12
+201	val_201	2008-04-09	12
+202	val_202	2008-04-09	12
+203	val_203	2008-04-09	12
+203	val_203	2008-04-09	12
+205	val_205	2008-04-09	12
+205	val_205	2008-04-09	12
+207	val_207	2008-04-09	12
+207	val_207	2008-04-09	12
+208	val_208	2008-04-09	12
+208	val_208	2008-04-09	12
+208	val_208	2008-04-09	12
+209	val_209	2008-04-09	12
+209	val_209	2008-04-09	12
+213	val_213	2008-04-09	12
+213	val_213	2008-04-09	12
+214	val_214	2008-04-09	12
+216	val_216	2008-04-09	12
+216	val_216	2008-04-09	12
+217	val_217	2008-04-09	12
+217	val_217	2008-04-09	12
+218	val_218	2008-04-09	12
+219	val_219	2008-04-09	12
+219	val_219	2008-04-09	12
+221	val_221	2008-04-09	12
+221	val_221	2008-04-09	12
+222	val_222	2008-04-09	12
+223	val_223	2008-04-09	12
+223	val_223	2008-04-09	12
+224	val_224	2008-04-09	12
+224	val_224	2008-04-09	12
+226	val_226	2008-04-09	12
+228	val_228	2008-04-09	12
+229	val_229	2008-04-09	12
+229	val_229	2008-04-09	12
+230	val_230	2008-04-09	12
+230	val_230	2008-04-09	12
+230	val_230	2008-04-09	12
+230	val_230	2008-04-09	12
+230	val_230	2008-04-09	12
+233	val_233	2008-04-09	12
+233	val_233	2008-04-09	12
+235	val_235	2008-04-09	12
+237	val_237	2008-04-09	12
+237	val_237	2008-04-09	12
+238	val_238	2008-04-09	12
+238	val_238	2008-04-09	12
+239	val_239	2008-04-09	12
+239	val_239	2008-04-09	12
+241	val_241	2008-04-09	12
+242	val_242	2008-04-09	12
+242	val_242	2008-04-09	12
+244	val_244	2008-04-09	12
+247	val_247	2008-04-09	12
+248	val_248	2008-04-09	12
+249	val_249	2008-04-09	12
+252	val_252	2008-04-09	12
+255	val_255	2008-04-09	12
+255	val_255	2008-04-09	12
+256	val_256	2008-04-09	12
+256	val_256	2008-04-09	12
+257	val_257	2008-04-09	12
+258	val_258	2008-04-09	12
+260	val_260	2008-04-09	12
+262	val_262	2008-04-09	12
+263	val_263	2008-04-09	12
+265	val_265	2008-04-09	12
+265	val_265	2008-04-09	12
+266	val_266	2008-04-09	12
+272	val_272	2008-04-09	12
+272	val_272	2008-04-09	12
+273	val_273	2008-04-09	12
+273	val_273	2008-04-09	12
+273	val_273	2008-04-09	12
+274	val_274	2008-04-09	12
+275	val_275	2008-04-09	12
+277	val_277	2008-04-09	12
+277	val_277	2008-04-09	12
+277	val_277	2008-04-09	12
+277	val_277	2008-04-09	12
+278	val_278	2008-04-09	12
+278	val_278	2008-04-09	12
+280	val_280	2008-04-09	12
+280	val_280	2008-04-09	12
+281	val_281	2008-04-09	12
+281	val_281	2008-04-09	12
+282	val_282	2008-04-09	12
+282	val_282	2008-04-09	12
+283	val_283	2008-04-09	12
+284	val_284	2008-04-09	12
+285	val_285	2008-04-09	12
+286	val_286	2008-04-09	12
+287	val_287	2008-04-09	12
+288	val_288	2008-04-09	12
+288	val_288	2008-04-09	12
+289	val_289	2008-04-09	12
+291	val_291	2008-04-09	12
+292	val_292	2008-04-09	12
+296	val_296	2008-04-09	12
+298	val_298	2008-04-09	12
+298	val_298	2008-04-09	12
+298	val_298	2008-04-09	12
+302	val_302	2008-04-09	12
+305	val_305	2008-04-09	12
+306	val_306	2008-04-09	12
+307	val_307	2008-04-09	12
+307	val_307	2008-04-09	12
+308	val_308	2008-04-09	12
+309	val_309	2008-04-09	12
+309	val_309	2008-04-09	12
+310	val_310	2008-04-09	12
+311	val_311	2008-04-09	12
+311	val_311	2008-04-09	12
+311	val_311	2008-04-09	12
+315	val_315	2008-04-09	12
+316	val_316	2008-04-09	12
+316	val_316	2008-04-09	12
+316	val_316	2008-04-09	12
+317	val_317	2008-04-09	12
+317	val_317	2008-04-09	12
+318	val_318	2008-04-09	12
+318	val_318	2008-04-09	12
+318	val_318	2008-04-09	12
+321	val_321	2008-04-09	12
+321	val_321	2008-04-09	12
+322	val_322	2008-04-09	12
+322	val_322	2008-04-09	12
+323	val_323	2008-04-09	12
+325	val_325	2008-04-09	12
+325	val_325	2008-04-09	12
+327	val_327	2008-04-09	12
+327	val_327	2008-04-09	12
+327	val_327	2008-04-09	12
+331	val_331	2008-04-09	12
+331	val_331	2008-04-09	12
+332	val_332	2008-04-09	12
+333	val_333	2008-04-09	12
+333	val_333	2008-04-09	12
+335	val_335	2008-04-09	12
+336	val_336	2008-04-09	12
+338	val_338	2008-04-09	12
+339	val_339	2008-04-09	12
+341	val_341	2008-04-09	12
+342	val_342	2008-04-09	12
+342	val_342	2008-04-09	12
+344	val_344	2008-04-09	12
+344	val_344	2008-04-09	12
+345	val_345	2008-04-09	12
+348	val_348	2008-04-09	12
+348	val_348	2008-04-09	12
+348	val_348	2008-04-09	12
+348	val_348	2008-04-09	12
+348	val_348	2008-04-09	12
+351	val_351	2008-04-09	12
+353	val_353	2008-04-09	12
+353	val_353	2008-04-09	12
+356	val_356	2008-04-09	12
+360	val_360	2008-04-09	12
+362	val_362	2008-04-09	12
+364	val_364	2008-04-09	12
+365	val_365	2008-04-09	12
+366	val_366	2008-04-09	12
+367	val_367	2008-04-09	12
+367	val_367	2008-04-09	12
+368	val_368	2008-04-09	12
+369	val_369	2008-04-09	12
+369	val_369	2008-04-09	12
+369	val_369	2008-04-09	12
+373	val_373	2008-04-09	12
+374	val_374	2008-04-09	12
+375	val_375	2008-04-09	12
+377	val_377	2008-04-09	12
+378	val_378	2008-04-09	12
+379	val_379	2008-04-09	12
+382	val_382	2008-04-09	12
+382	val_382	2008-04-09	12
+384	val_384	2008-04-09	12
+384	val_384	2008-04-09	12
+384	val_384	2008-04-09	12
+386	val_386	2008-04-09	12
+389	val_389	2008-04-09	12
+392	val_392	2008-04-09	12
+393	val_393	2008-04-09	12
+394	val_394	2008-04-09	12
+395	val_395	2008-04-09	12
+395	val_395	2008-04-09	12
+396	val_396	2008-04-09	12
+396	val_396	2008-04-09	12
+396	val_396	2008-04-09	12
+397	val_397	2008-04-09	12
+397	val_397	2008-04-09	12
+399	val_399	2008-04-09	12
+399	val_399	2008-04-09	12
+400	val_400	2008-04-09	12
+401	val_401	2008-04-09	12
+401	val_401	2008-04-09	12
+401	val_401	2008-04-09	12
+401	val_401	2008-04-09	12
+401	val_401	2008-04-09	12
+402	val_402	2008-04-09	12
+403	val_403	2008-04-09	12
+403	val_403	2008-04-09	12
+403	val_403	2008-04-09	12
+404	val_404	2008-04-09	12
+404	val_404	2008-04-09	12
+406	val_406	2008-04-09	12
+406	val_406	2008-04-09	12
+406	val_406	2008-04-09	12
+406	val_406	2008-04-09	12
+407	val_407	2008-04-09	12
+409	val_409	2008-04-09	12
+409	val_409	2008-04-09	12
+409	val_409	2008-04-09	12
+411	val_411	2008-04-09	12
+413	val_413	2008-04-09	12
+413	val_413	2008-04-09	12
+414	val_414	2008-04-09	12
+414	val_414	2008-04-09	12
+417	val_417	2008-04-09	12
+417	val_417	2008-04-09	12
+417	val_417	2008-04-09	12
+418	val_418	2008-04-09	12
+419	val_419	2008-04-09	12
+421	val_421	2008-04-09	12
+424	val_424	2008-04-09	12
+424	val_424	2008-04-09	12
+427	val_427	2008-04-09	12
+429	val_429	2008-04-09	12
+429	val_429	2008-04-09	12
+430	val_430	2008-04-09	12
+430	val_430	2008-04-09	12
+430	val_430	2008-04-09	12
+431	val_431	2008-04-09	12
+431	val_431	2008-04-09	12
+431	val_431	2008-04-09	12
+432	val_432	2008-04-09	12
+435	val_435	2008-04-09	12
+436	val_436	2008-04-09	12
+437	val_437	2008-04-09	12
+438	val_438	2008-04-09	12
+438	val_438	2008-04-09	12
+438	val_438	2008-04-09	12
+439	val_439	2008-04-09	12
+439	val_439	2008-04-09	12
+443	val_443	2008-04-09	12
+444	val_444	2008-04-09	12
+446	val_446	2008-04-09	12
+448	val_448	2008-04-09	12
+449	val_449	2008-04-09	12
+452	val_452	2008-04-09	12
+453	val_453	2008-04-09	12
+454	val_454	2008-04-09	12
+454	val_454	2008-04-09	12
+454	val_454	2008-04-09	12
+455	val_455	2008-04-09	12
+457	val_457	2008-04-09	12
+458	val_458	2008-04-09	12
+458	val_458	2008-04-09	12
+459	val_459	2008-04-09	12
+459	val_459	2008-04-09	12
+460	val_460	2008-04-09	12
+462	val_462	2008-04-09	12
+462	val_462	2008-04-09	12
+463	val_463	2008-04-09	12
+463	val_463	2008-04-09	12
+466	val_466	2008-04-09	12
+466	val_466	2008-04-09	12
+466	val_466	2008-04-09	12
+467	val_467	2008-04-09	12
+468	val_468	2008-04-09	12
+468	val_468	2008-04-09	12
+468	val_468	2008-04-09	12
+468	val_468	2008-04-09	12
+469	val_469	2008-04-09	12
+469	val_469	2008-04-09	12
+469	val_469	2008-04-09	12
+469	val_469	2008-04-09	12
+469	val_469	2008-04-09	12
+470	val_470	2008-04-09	12
+472	val_472	2008-04-09	12
+475	val_475	2008-04-09	12
+477	val_477	2008-04-09	12
+478	val_478	2008-04-09	12
+478	val_478	2008-04-09	12
+479	val_479	2008-04-09	12
+480	val_480	2008-04-09	12
+480	val_480	2008-04-09	12
+480	val_480	2008-04-09	12
+481	val_481	2008-04-09	12
+482	val_482	2008-04-09	12
+483	val_483	2008-04-09	12
+484	val_484	2008-04-09	12
+485	val_485	2008-04-09	12
+487	val_487	2008-04-09	12
+489	val_489	2008-04-09	12
+489	val_489	2008-04-09	12
+489	val_489	2008-04-09	12
+489	val_489	2008-04-09	12
+490	val_490	2008-04-09	12
+491	val_491	2008-04-09	12
+492	val_492	2008-04-09	12
+492	val_492	2008-04-09	12
+493	val_493	2008-04-09	12
+494	val_494	2008-04-09	12
+495	val_495	2008-04-09	12
+496	val_496	2008-04-09	12
+497	val_497	2008-04-09	12
+498	val_498	2008-04-09	12
+498	val_498	2008-04-09	12
+498	val_498	2008-04-09	12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd b/sql/hive/src/test/resources/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd
index 8017948fc5f2f5c7622ff400f2e4368d560a2414..3951005f24d2fa2c4aee86584b3f63ffad42a8ea 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd
+++ b/sql/hive/src/test/resources/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd
@@ -1,12 +1,12 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-hr                  	string              	None                
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part4, dbName:default, owner:marmbrus, createTime:1390899619, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part4, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{p3=v3, transient_lastDdlTime=1390899619}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:nzhang_part4, dbName:default, owner:marmbrus, createTime:1413887619, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part4, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887619}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b b/sql/hive/src/test/resources/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b
index 3f5f6b6f83715e18e5ce5111bcd9ca86b0877f9a..6f2ec094839891d6c2babf6312d89f146d6308f6 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b
+++ b/sql/hive/src/test/resources/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b
@@ -1,9 +1,9 @@
-key                 	string              	None                
-value               	string              	None                
+key                 	string              	                    
+value               	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-value               	string              	None                
+value               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part5, dbName:default, owner:marmbrus, createTime:1390899637, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part5, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:value, type:string, comment:null)], parameters:{p3=v3, transient_lastDdlTime=1390899637}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:nzhang_part5, dbName:default, owner:marmbrus, createTime:1413887658, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part5, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:value, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887658}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d b/sql/hive/src/test/resources/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d
index 4b9e5b31f440160dd40bb88b69d55a66c820785a..3d4fc3e4d1fb19b93ee84a6ce3955fcd76f2c0ad 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d
+++ b/sql/hive/src/test/resources/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d
@@ -1,12 +1,12 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-hr                  	string              	None                
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part6, dbName:default, owner:marmbrus, createTime:1390899654, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part6, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{p3=v3, transient_lastDdlTime=1390899654}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:nzhang_part6, dbName:default, owner:marmbrus, createTime:1413887681, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part6, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887681}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a b/sql/hive/src/test/resources/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a
index d62156d392d5e3a481d447d02fa8974a5838c797..be50a6974cd82d8cdda824d318b8f6e4dd401140 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a
+++ b/sql/hive/src/test/resources/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a
@@ -1,12 +1,12 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-hr                  	string              	None                
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part7, dbName:default, owner:marmbrus, createTime:1390899664, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part7, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{p3=v3, transient_lastDdlTime=1390899664}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:nzhang_part7, dbName:default, owner:marmbrus, createTime:1413887689, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part7, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887689}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a b/sql/hive/src/test/resources/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a
index 91fa51871a4b87066a9e7db4af4e26089263b973..5496973d9b2eb773f8cfced58f98525dfc6e9d85 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a
+++ b/sql/hive/src/test/resources/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a
@@ -1,12 +1,12 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-hr                  	string              	None                
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part8, dbName:default, owner:marmbrus, createTime:1390899674, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part8, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{p3=v3, transient_lastDdlTime=1390899674}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:nzhang_part8, dbName:default, owner:marmbrus, createTime:1413887697, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part8, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887697}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part8-9-704bd110d9aaa2ac678b7fbf645abdb9 b/sql/hive/src/test/resources/golden/load_dyn_part8-9-704bd110d9aaa2ac678b7fbf645abdb9
new file mode 100644
index 0000000000000000000000000000000000000000..c59426f6c153e7a0c57c2f7a116a0f189f20fbb2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/load_dyn_part8-9-704bd110d9aaa2ac678b7fbf645abdb9
@@ -0,0 +1,2000 @@
+0	val_0	2008-04-08	11
+0	val_0	2008-04-08	11
+0	val_0	2008-04-08	11
+2	val_2	2008-04-08	11
+4	val_4	2008-04-08	11
+5	val_5	2008-04-08	11
+5	val_5	2008-04-08	11
+5	val_5	2008-04-08	11
+8	val_8	2008-04-08	11
+9	val_9	2008-04-08	11
+10	val_10	2008-04-08	11
+11	val_11	2008-04-08	11
+12	val_12	2008-04-08	11
+12	val_12	2008-04-08	11
+15	val_15	2008-04-08	11
+15	val_15	2008-04-08	11
+17	val_17	2008-04-08	11
+18	val_18	2008-04-08	11
+18	val_18	2008-04-08	11
+19	val_19	2008-04-08	11
+20	val_20	2008-04-08	11
+24	val_24	2008-04-08	11
+24	val_24	2008-04-08	11
+26	val_26	2008-04-08	11
+26	val_26	2008-04-08	11
+27	val_27	2008-04-08	11
+28	val_28	2008-04-08	11
+30	val_30	2008-04-08	11
+33	val_33	2008-04-08	11
+34	val_34	2008-04-08	11
+35	val_35	2008-04-08	11
+35	val_35	2008-04-08	11
+35	val_35	2008-04-08	11
+37	val_37	2008-04-08	11
+37	val_37	2008-04-08	11
+41	val_41	2008-04-08	11
+42	val_42	2008-04-08	11
+42	val_42	2008-04-08	11
+43	val_43	2008-04-08	11
+44	val_44	2008-04-08	11
+47	val_47	2008-04-08	11
+51	val_51	2008-04-08	11
+51	val_51	2008-04-08	11
+53	val_53	2008-04-08	11
+54	val_54	2008-04-08	11
+57	val_57	2008-04-08	11
+58	val_58	2008-04-08	11
+58	val_58	2008-04-08	11
+64	val_64	2008-04-08	11
+65	val_65	2008-04-08	11
+66	val_66	2008-04-08	11
+67	val_67	2008-04-08	11
+67	val_67	2008-04-08	11
+69	val_69	2008-04-08	11
+70	val_70	2008-04-08	11
+70	val_70	2008-04-08	11
+70	val_70	2008-04-08	11
+72	val_72	2008-04-08	11
+72	val_72	2008-04-08	11
+74	val_74	2008-04-08	11
+76	val_76	2008-04-08	11
+76	val_76	2008-04-08	11
+77	val_77	2008-04-08	11
+78	val_78	2008-04-08	11
+80	val_80	2008-04-08	11
+82	val_82	2008-04-08	11
+83	val_83	2008-04-08	11
+83	val_83	2008-04-08	11
+84	val_84	2008-04-08	11
+84	val_84	2008-04-08	11
+85	val_85	2008-04-08	11
+86	val_86	2008-04-08	11
+87	val_87	2008-04-08	11
+90	val_90	2008-04-08	11
+90	val_90	2008-04-08	11
+90	val_90	2008-04-08	11
+92	val_92	2008-04-08	11
+95	val_95	2008-04-08	11
+95	val_95	2008-04-08	11
+96	val_96	2008-04-08	11
+97	val_97	2008-04-08	11
+97	val_97	2008-04-08	11
+98	val_98	2008-04-08	11
+98	val_98	2008-04-08	11
+100	val_100	2008-04-08	11
+100	val_100	2008-04-08	11
+103	val_103	2008-04-08	11
+103	val_103	2008-04-08	11
+104	val_104	2008-04-08	11
+104	val_104	2008-04-08	11
+105	val_105	2008-04-08	11
+111	val_111	2008-04-08	11
+113	val_113	2008-04-08	11
+113	val_113	2008-04-08	11
+114	val_114	2008-04-08	11
+116	val_116	2008-04-08	11
+118	val_118	2008-04-08	11
+118	val_118	2008-04-08	11
+119	val_119	2008-04-08	11
+119	val_119	2008-04-08	11
+119	val_119	2008-04-08	11
+120	val_120	2008-04-08	11
+120	val_120	2008-04-08	11
+125	val_125	2008-04-08	11
+125	val_125	2008-04-08	11
+126	val_126	2008-04-08	11
+128	val_128	2008-04-08	11
+128	val_128	2008-04-08	11
+128	val_128	2008-04-08	11
+129	val_129	2008-04-08	11
+129	val_129	2008-04-08	11
+131	val_131	2008-04-08	11
+133	val_133	2008-04-08	11
+134	val_134	2008-04-08	11
+134	val_134	2008-04-08	11
+136	val_136	2008-04-08	11
+137	val_137	2008-04-08	11
+137	val_137	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+143	val_143	2008-04-08	11
+145	val_145	2008-04-08	11
+146	val_146	2008-04-08	11
+146	val_146	2008-04-08	11
+149	val_149	2008-04-08	11
+149	val_149	2008-04-08	11
+150	val_150	2008-04-08	11
+152	val_152	2008-04-08	11
+152	val_152	2008-04-08	11
+153	val_153	2008-04-08	11
+155	val_155	2008-04-08	11
+156	val_156	2008-04-08	11
+157	val_157	2008-04-08	11
+158	val_158	2008-04-08	11
+160	val_160	2008-04-08	11
+162	val_162	2008-04-08	11
+163	val_163	2008-04-08	11
+164	val_164	2008-04-08	11
+164	val_164	2008-04-08	11
+165	val_165	2008-04-08	11
+165	val_165	2008-04-08	11
+166	val_166	2008-04-08	11
+167	val_167	2008-04-08	11
+167	val_167	2008-04-08	11
+167	val_167	2008-04-08	11
+168	val_168	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+170	val_170	2008-04-08	11
+172	val_172	2008-04-08	11
+172	val_172	2008-04-08	11
+174	val_174	2008-04-08	11
+174	val_174	2008-04-08	11
+175	val_175	2008-04-08	11
+175	val_175	2008-04-08	11
+176	val_176	2008-04-08	11
+176	val_176	2008-04-08	11
+177	val_177	2008-04-08	11
+178	val_178	2008-04-08	11
+179	val_179	2008-04-08	11
+179	val_179	2008-04-08	11
+180	val_180	2008-04-08	11
+181	val_181	2008-04-08	11
+183	val_183	2008-04-08	11
+186	val_186	2008-04-08	11
+187	val_187	2008-04-08	11
+187	val_187	2008-04-08	11
+187	val_187	2008-04-08	11
+189	val_189	2008-04-08	11
+190	val_190	2008-04-08	11
+191	val_191	2008-04-08	11
+191	val_191	2008-04-08	11
+192	val_192	2008-04-08	11
+193	val_193	2008-04-08	11
+193	val_193	2008-04-08	11
+193	val_193	2008-04-08	11
+194	val_194	2008-04-08	11
+195	val_195	2008-04-08	11
+195	val_195	2008-04-08	11
+196	val_196	2008-04-08	11
+197	val_197	2008-04-08	11
+197	val_197	2008-04-08	11
+199	val_199	2008-04-08	11
+199	val_199	2008-04-08	11
+199	val_199	2008-04-08	11
+200	val_200	2008-04-08	11
+200	val_200	2008-04-08	11
+201	val_201	2008-04-08	11
+202	val_202	2008-04-08	11
+203	val_203	2008-04-08	11
+203	val_203	2008-04-08	11
+205	val_205	2008-04-08	11
+205	val_205	2008-04-08	11
+207	val_207	2008-04-08	11
+207	val_207	2008-04-08	11
+208	val_208	2008-04-08	11
+208	val_208	2008-04-08	11
+208	val_208	2008-04-08	11
+209	val_209	2008-04-08	11
+209	val_209	2008-04-08	11
+213	val_213	2008-04-08	11
+213	val_213	2008-04-08	11
+214	val_214	2008-04-08	11
+216	val_216	2008-04-08	11
+216	val_216	2008-04-08	11
+217	val_217	2008-04-08	11
+217	val_217	2008-04-08	11
+218	val_218	2008-04-08	11
+219	val_219	2008-04-08	11
+219	val_219	2008-04-08	11
+221	val_221	2008-04-08	11
+221	val_221	2008-04-08	11
+222	val_222	2008-04-08	11
+223	val_223	2008-04-08	11
+223	val_223	2008-04-08	11
+224	val_224	2008-04-08	11
+224	val_224	2008-04-08	11
+226	val_226	2008-04-08	11
+228	val_228	2008-04-08	11
+229	val_229	2008-04-08	11
+229	val_229	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+233	val_233	2008-04-08	11
+233	val_233	2008-04-08	11
+235	val_235	2008-04-08	11
+237	val_237	2008-04-08	11
+237	val_237	2008-04-08	11
+238	val_238	2008-04-08	11
+238	val_238	2008-04-08	11
+239	val_239	2008-04-08	11
+239	val_239	2008-04-08	11
+241	val_241	2008-04-08	11
+242	val_242	2008-04-08	11
+242	val_242	2008-04-08	11
+244	val_244	2008-04-08	11
+247	val_247	2008-04-08	11
+248	val_248	2008-04-08	11
+249	val_249	2008-04-08	11
+252	val_252	2008-04-08	11
+255	val_255	2008-04-08	11
+255	val_255	2008-04-08	11
+256	val_256	2008-04-08	11
+256	val_256	2008-04-08	11
+257	val_257	2008-04-08	11
+258	val_258	2008-04-08	11
+260	val_260	2008-04-08	11
+262	val_262	2008-04-08	11
+263	val_263	2008-04-08	11
+265	val_265	2008-04-08	11
+265	val_265	2008-04-08	11
+266	val_266	2008-04-08	11
+272	val_272	2008-04-08	11
+272	val_272	2008-04-08	11
+273	val_273	2008-04-08	11
+273	val_273	2008-04-08	11
+273	val_273	2008-04-08	11
+274	val_274	2008-04-08	11
+275	val_275	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+278	val_278	2008-04-08	11
+278	val_278	2008-04-08	11
+280	val_280	2008-04-08	11
+280	val_280	2008-04-08	11
+281	val_281	2008-04-08	11
+281	val_281	2008-04-08	11
+282	val_282	2008-04-08	11
+282	val_282	2008-04-08	11
+283	val_283	2008-04-08	11
+284	val_284	2008-04-08	11
+285	val_285	2008-04-08	11
+286	val_286	2008-04-08	11
+287	val_287	2008-04-08	11
+288	val_288	2008-04-08	11
+288	val_288	2008-04-08	11
+289	val_289	2008-04-08	11
+291	val_291	2008-04-08	11
+292	val_292	2008-04-08	11
+296	val_296	2008-04-08	11
+298	val_298	2008-04-08	11
+298	val_298	2008-04-08	11
+298	val_298	2008-04-08	11
+302	val_302	2008-04-08	11
+305	val_305	2008-04-08	11
+306	val_306	2008-04-08	11
+307	val_307	2008-04-08	11
+307	val_307	2008-04-08	11
+308	val_308	2008-04-08	11
+309	val_309	2008-04-08	11
+309	val_309	2008-04-08	11
+310	val_310	2008-04-08	11
+311	val_311	2008-04-08	11
+311	val_311	2008-04-08	11
+311	val_311	2008-04-08	11
+315	val_315	2008-04-08	11
+316	val_316	2008-04-08	11
+316	val_316	2008-04-08	11
+316	val_316	2008-04-08	11
+317	val_317	2008-04-08	11
+317	val_317	2008-04-08	11
+318	val_318	2008-04-08	11
+318	val_318	2008-04-08	11
+318	val_318	2008-04-08	11
+321	val_321	2008-04-08	11
+321	val_321	2008-04-08	11
+322	val_322	2008-04-08	11
+322	val_322	2008-04-08	11
+323	val_323	2008-04-08	11
+325	val_325	2008-04-08	11
+325	val_325	2008-04-08	11
+327	val_327	2008-04-08	11
+327	val_327	2008-04-08	11
+327	val_327	2008-04-08	11
+331	val_331	2008-04-08	11
+331	val_331	2008-04-08	11
+332	val_332	2008-04-08	11
+333	val_333	2008-04-08	11
+333	val_333	2008-04-08	11
+335	val_335	2008-04-08	11
+336	val_336	2008-04-08	11
+338	val_338	2008-04-08	11
+339	val_339	2008-04-08	11
+341	val_341	2008-04-08	11
+342	val_342	2008-04-08	11
+342	val_342	2008-04-08	11
+344	val_344	2008-04-08	11
+344	val_344	2008-04-08	11
+345	val_345	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+351	val_351	2008-04-08	11
+353	val_353	2008-04-08	11
+353	val_353	2008-04-08	11
+356	val_356	2008-04-08	11
+360	val_360	2008-04-08	11
+362	val_362	2008-04-08	11
+364	val_364	2008-04-08	11
+365	val_365	2008-04-08	11
+366	val_366	2008-04-08	11
+367	val_367	2008-04-08	11
+367	val_367	2008-04-08	11
+368	val_368	2008-04-08	11
+369	val_369	2008-04-08	11
+369	val_369	2008-04-08	11
+369	val_369	2008-04-08	11
+373	val_373	2008-04-08	11
+374	val_374	2008-04-08	11
+375	val_375	2008-04-08	11
+377	val_377	2008-04-08	11
+378	val_378	2008-04-08	11
+379	val_379	2008-04-08	11
+382	val_382	2008-04-08	11
+382	val_382	2008-04-08	11
+384	val_384	2008-04-08	11
+384	val_384	2008-04-08	11
+384	val_384	2008-04-08	11
+386	val_386	2008-04-08	11
+389	val_389	2008-04-08	11
+392	val_392	2008-04-08	11
+393	val_393	2008-04-08	11
+394	val_394	2008-04-08	11
+395	val_395	2008-04-08	11
+395	val_395	2008-04-08	11
+396	val_396	2008-04-08	11
+396	val_396	2008-04-08	11
+396	val_396	2008-04-08	11
+397	val_397	2008-04-08	11
+397	val_397	2008-04-08	11
+399	val_399	2008-04-08	11
+399	val_399	2008-04-08	11
+400	val_400	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+402	val_402	2008-04-08	11
+403	val_403	2008-04-08	11
+403	val_403	2008-04-08	11
+403	val_403	2008-04-08	11
+404	val_404	2008-04-08	11
+404	val_404	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+407	val_407	2008-04-08	11
+409	val_409	2008-04-08	11
+409	val_409	2008-04-08	11
+409	val_409	2008-04-08	11
+411	val_411	2008-04-08	11
+413	val_413	2008-04-08	11
+413	val_413	2008-04-08	11
+414	val_414	2008-04-08	11
+414	val_414	2008-04-08	11
+417	val_417	2008-04-08	11
+417	val_417	2008-04-08	11
+417	val_417	2008-04-08	11
+418	val_418	2008-04-08	11
+419	val_419	2008-04-08	11
+421	val_421	2008-04-08	11
+424	val_424	2008-04-08	11
+424	val_424	2008-04-08	11
+427	val_427	2008-04-08	11
+429	val_429	2008-04-08	11
+429	val_429	2008-04-08	11
+430	val_430	2008-04-08	11
+430	val_430	2008-04-08	11
+430	val_430	2008-04-08	11
+431	val_431	2008-04-08	11
+431	val_431	2008-04-08	11
+431	val_431	2008-04-08	11
+432	val_432	2008-04-08	11
+435	val_435	2008-04-08	11
+436	val_436	2008-04-08	11
+437	val_437	2008-04-08	11
+438	val_438	2008-04-08	11
+438	val_438	2008-04-08	11
+438	val_438	2008-04-08	11
+439	val_439	2008-04-08	11
+439	val_439	2008-04-08	11
+443	val_443	2008-04-08	11
+444	val_444	2008-04-08	11
+446	val_446	2008-04-08	11
+448	val_448	2008-04-08	11
+449	val_449	2008-04-08	11
+452	val_452	2008-04-08	11
+453	val_453	2008-04-08	11
+454	val_454	2008-04-08	11
+454	val_454	2008-04-08	11
+454	val_454	2008-04-08	11
+455	val_455	2008-04-08	11
+457	val_457	2008-04-08	11
+458	val_458	2008-04-08	11
+458	val_458	2008-04-08	11
+459	val_459	2008-04-08	11
+459	val_459	2008-04-08	11
+460	val_460	2008-04-08	11
+462	val_462	2008-04-08	11
+462	val_462	2008-04-08	11
+463	val_463	2008-04-08	11
+463	val_463	2008-04-08	11
+466	val_466	2008-04-08	11
+466	val_466	2008-04-08	11
+466	val_466	2008-04-08	11
+467	val_467	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+470	val_470	2008-04-08	11
+472	val_472	2008-04-08	11
+475	val_475	2008-04-08	11
+477	val_477	2008-04-08	11
+478	val_478	2008-04-08	11
+478	val_478	2008-04-08	11
+479	val_479	2008-04-08	11
+480	val_480	2008-04-08	11
+480	val_480	2008-04-08	11
+480	val_480	2008-04-08	11
+481	val_481	2008-04-08	11
+482	val_482	2008-04-08	11
+483	val_483	2008-04-08	11
+484	val_484	2008-04-08	11
+485	val_485	2008-04-08	11
+487	val_487	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+490	val_490	2008-04-08	11
+491	val_491	2008-04-08	11
+492	val_492	2008-04-08	11
+492	val_492	2008-04-08	11
+493	val_493	2008-04-08	11
+494	val_494	2008-04-08	11
+495	val_495	2008-04-08	11
+496	val_496	2008-04-08	11
+497	val_497	2008-04-08	11
+498	val_498	2008-04-08	11
+498	val_498	2008-04-08	11
+498	val_498	2008-04-08	11
+0	val_0	2008-04-08	12
+0	val_0	2008-04-08	12
+0	val_0	2008-04-08	12
+2	val_2	2008-04-08	12
+4	val_4	2008-04-08	12
+5	val_5	2008-04-08	12
+5	val_5	2008-04-08	12
+5	val_5	2008-04-08	12
+8	val_8	2008-04-08	12
+9	val_9	2008-04-08	12
+10	val_10	2008-04-08	12
+11	val_11	2008-04-08	12
+12	val_12	2008-04-08	12
+12	val_12	2008-04-08	12
+15	val_15	2008-04-08	12
+15	val_15	2008-04-08	12
+17	val_17	2008-04-08	12
+18	val_18	2008-04-08	12
+18	val_18	2008-04-08	12
+19	val_19	2008-04-08	12
+20	val_20	2008-04-08	12
+24	val_24	2008-04-08	12
+24	val_24	2008-04-08	12
+26	val_26	2008-04-08	12
+26	val_26	2008-04-08	12
+27	val_27	2008-04-08	12
+28	val_28	2008-04-08	12
+30	val_30	2008-04-08	12
+33	val_33	2008-04-08	12
+34	val_34	2008-04-08	12
+35	val_35	2008-04-08	12
+35	val_35	2008-04-08	12
+35	val_35	2008-04-08	12
+37	val_37	2008-04-08	12
+37	val_37	2008-04-08	12
+41	val_41	2008-04-08	12
+42	val_42	2008-04-08	12
+42	val_42	2008-04-08	12
+43	val_43	2008-04-08	12
+44	val_44	2008-04-08	12
+47	val_47	2008-04-08	12
+51	val_51	2008-04-08	12
+51	val_51	2008-04-08	12
+53	val_53	2008-04-08	12
+54	val_54	2008-04-08	12
+57	val_57	2008-04-08	12
+58	val_58	2008-04-08	12
+58	val_58	2008-04-08	12
+64	val_64	2008-04-08	12
+65	val_65	2008-04-08	12
+66	val_66	2008-04-08	12
+67	val_67	2008-04-08	12
+67	val_67	2008-04-08	12
+69	val_69	2008-04-08	12
+70	val_70	2008-04-08	12
+70	val_70	2008-04-08	12
+70	val_70	2008-04-08	12
+72	val_72	2008-04-08	12
+72	val_72	2008-04-08	12
+74	val_74	2008-04-08	12
+76	val_76	2008-04-08	12
+76	val_76	2008-04-08	12
+77	val_77	2008-04-08	12
+78	val_78	2008-04-08	12
+80	val_80	2008-04-08	12
+82	val_82	2008-04-08	12
+83	val_83	2008-04-08	12
+83	val_83	2008-04-08	12
+84	val_84	2008-04-08	12
+84	val_84	2008-04-08	12
+85	val_85	2008-04-08	12
+86	val_86	2008-04-08	12
+87	val_87	2008-04-08	12
+90	val_90	2008-04-08	12
+90	val_90	2008-04-08	12
+90	val_90	2008-04-08	12
+92	val_92	2008-04-08	12
+95	val_95	2008-04-08	12
+95	val_95	2008-04-08	12
+96	val_96	2008-04-08	12
+97	val_97	2008-04-08	12
+97	val_97	2008-04-08	12
+98	val_98	2008-04-08	12
+98	val_98	2008-04-08	12
+100	val_100	2008-04-08	12
+100	val_100	2008-04-08	12
+103	val_103	2008-04-08	12
+103	val_103	2008-04-08	12
+104	val_104	2008-04-08	12
+104	val_104	2008-04-08	12
+105	val_105	2008-04-08	12
+111	val_111	2008-04-08	12
+113	val_113	2008-04-08	12
+113	val_113	2008-04-08	12
+114	val_114	2008-04-08	12
+116	val_116	2008-04-08	12
+118	val_118	2008-04-08	12
+118	val_118	2008-04-08	12
+119	val_119	2008-04-08	12
+119	val_119	2008-04-08	12
+119	val_119	2008-04-08	12
+120	val_120	2008-04-08	12
+120	val_120	2008-04-08	12
+125	val_125	2008-04-08	12
+125	val_125	2008-04-08	12
+126	val_126	2008-04-08	12
+128	val_128	2008-04-08	12
+128	val_128	2008-04-08	12
+128	val_128	2008-04-08	12
+129	val_129	2008-04-08	12
+129	val_129	2008-04-08	12
+131	val_131	2008-04-08	12
+133	val_133	2008-04-08	12
+134	val_134	2008-04-08	12
+134	val_134	2008-04-08	12
+136	val_136	2008-04-08	12
+137	val_137	2008-04-08	12
+137	val_137	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+143	val_143	2008-04-08	12
+145	val_145	2008-04-08	12
+146	val_146	2008-04-08	12
+146	val_146	2008-04-08	12
+149	val_149	2008-04-08	12
+149	val_149	2008-04-08	12
+150	val_150	2008-04-08	12
+152	val_152	2008-04-08	12
+152	val_152	2008-04-08	12
+153	val_153	2008-04-08	12
+155	val_155	2008-04-08	12
+156	val_156	2008-04-08	12
+157	val_157	2008-04-08	12
+158	val_158	2008-04-08	12
+160	val_160	2008-04-08	12
+162	val_162	2008-04-08	12
+163	val_163	2008-04-08	12
+164	val_164	2008-04-08	12
+164	val_164	2008-04-08	12
+165	val_165	2008-04-08	12
+165	val_165	2008-04-08	12
+166	val_166	2008-04-08	12
+167	val_167	2008-04-08	12
+167	val_167	2008-04-08	12
+167	val_167	2008-04-08	12
+168	val_168	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+170	val_170	2008-04-08	12
+172	val_172	2008-04-08	12
+172	val_172	2008-04-08	12
+174	val_174	2008-04-08	12
+174	val_174	2008-04-08	12
+175	val_175	2008-04-08	12
+175	val_175	2008-04-08	12
+176	val_176	2008-04-08	12
+176	val_176	2008-04-08	12
+177	val_177	2008-04-08	12
+178	val_178	2008-04-08	12
+179	val_179	2008-04-08	12
+179	val_179	2008-04-08	12
+180	val_180	2008-04-08	12
+181	val_181	2008-04-08	12
+183	val_183	2008-04-08	12
+186	val_186	2008-04-08	12
+187	val_187	2008-04-08	12
+187	val_187	2008-04-08	12
+187	val_187	2008-04-08	12
+189	val_189	2008-04-08	12
+190	val_190	2008-04-08	12
+191	val_191	2008-04-08	12
+191	val_191	2008-04-08	12
+192	val_192	2008-04-08	12
+193	val_193	2008-04-08	12
+193	val_193	2008-04-08	12
+193	val_193	2008-04-08	12
+194	val_194	2008-04-08	12
+195	val_195	2008-04-08	12
+195	val_195	2008-04-08	12
+196	val_196	2008-04-08	12
+197	val_197	2008-04-08	12
+197	val_197	2008-04-08	12
+199	val_199	2008-04-08	12
+199	val_199	2008-04-08	12
+199	val_199	2008-04-08	12
+200	val_200	2008-04-08	12
+200	val_200	2008-04-08	12
+201	val_201	2008-04-08	12
+202	val_202	2008-04-08	12
+203	val_203	2008-04-08	12
+203	val_203	2008-04-08	12
+205	val_205	2008-04-08	12
+205	val_205	2008-04-08	12
+207	val_207	2008-04-08	12
+207	val_207	2008-04-08	12
+208	val_208	2008-04-08	12
+208	val_208	2008-04-08	12
+208	val_208	2008-04-08	12
+209	val_209	2008-04-08	12
+209	val_209	2008-04-08	12
+213	val_213	2008-04-08	12
+213	val_213	2008-04-08	12
+214	val_214	2008-04-08	12
+216	val_216	2008-04-08	12
+216	val_216	2008-04-08	12
+217	val_217	2008-04-08	12
+217	val_217	2008-04-08	12
+218	val_218	2008-04-08	12
+219	val_219	2008-04-08	12
+219	val_219	2008-04-08	12
+221	val_221	2008-04-08	12
+221	val_221	2008-04-08	12
+222	val_222	2008-04-08	12
+223	val_223	2008-04-08	12
+223	val_223	2008-04-08	12
+224	val_224	2008-04-08	12
+224	val_224	2008-04-08	12
+226	val_226	2008-04-08	12
+228	val_228	2008-04-08	12
+229	val_229	2008-04-08	12
+229	val_229	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+233	val_233	2008-04-08	12
+233	val_233	2008-04-08	12
+235	val_235	2008-04-08	12
+237	val_237	2008-04-08	12
+237	val_237	2008-04-08	12
+238	val_238	2008-04-08	12
+238	val_238	2008-04-08	12
+239	val_239	2008-04-08	12
+239	val_239	2008-04-08	12
+241	val_241	2008-04-08	12
+242	val_242	2008-04-08	12
+242	val_242	2008-04-08	12
+244	val_244	2008-04-08	12
+247	val_247	2008-04-08	12
+248	val_248	2008-04-08	12
+249	val_249	2008-04-08	12
+252	val_252	2008-04-08	12
+255	val_255	2008-04-08	12
+255	val_255	2008-04-08	12
+256	val_256	2008-04-08	12
+256	val_256	2008-04-08	12
+257	val_257	2008-04-08	12
+258	val_258	2008-04-08	12
+260	val_260	2008-04-08	12
+262	val_262	2008-04-08	12
+263	val_263	2008-04-08	12
+265	val_265	2008-04-08	12
+265	val_265	2008-04-08	12
+266	val_266	2008-04-08	12
+272	val_272	2008-04-08	12
+272	val_272	2008-04-08	12
+273	val_273	2008-04-08	12
+273	val_273	2008-04-08	12
+273	val_273	2008-04-08	12
+274	val_274	2008-04-08	12
+275	val_275	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+278	val_278	2008-04-08	12
+278	val_278	2008-04-08	12
+280	val_280	2008-04-08	12
+280	val_280	2008-04-08	12
+281	val_281	2008-04-08	12
+281	val_281	2008-04-08	12
+282	val_282	2008-04-08	12
+282	val_282	2008-04-08	12
+283	val_283	2008-04-08	12
+284	val_284	2008-04-08	12
+285	val_285	2008-04-08	12
+286	val_286	2008-04-08	12
+287	val_287	2008-04-08	12
+288	val_288	2008-04-08	12
+288	val_288	2008-04-08	12
+289	val_289	2008-04-08	12
+291	val_291	2008-04-08	12
+292	val_292	2008-04-08	12
+296	val_296	2008-04-08	12
+298	val_298	2008-04-08	12
+298	val_298	2008-04-08	12
+298	val_298	2008-04-08	12
+302	val_302	2008-04-08	12
+305	val_305	2008-04-08	12
+306	val_306	2008-04-08	12
+307	val_307	2008-04-08	12
+307	val_307	2008-04-08	12
+308	val_308	2008-04-08	12
+309	val_309	2008-04-08	12
+309	val_309	2008-04-08	12
+310	val_310	2008-04-08	12
+311	val_311	2008-04-08	12
+311	val_311	2008-04-08	12
+311	val_311	2008-04-08	12
+315	val_315	2008-04-08	12
+316	val_316	2008-04-08	12
+316	val_316	2008-04-08	12
+316	val_316	2008-04-08	12
+317	val_317	2008-04-08	12
+317	val_317	2008-04-08	12
+318	val_318	2008-04-08	12
+318	val_318	2008-04-08	12
+318	val_318	2008-04-08	12
+321	val_321	2008-04-08	12
+321	val_321	2008-04-08	12
+322	val_322	2008-04-08	12
+322	val_322	2008-04-08	12
+323	val_323	2008-04-08	12
+325	val_325	2008-04-08	12
+325	val_325	2008-04-08	12
+327	val_327	2008-04-08	12
+327	val_327	2008-04-08	12
+327	val_327	2008-04-08	12
+331	val_331	2008-04-08	12
+331	val_331	2008-04-08	12
+332	val_332	2008-04-08	12
+333	val_333	2008-04-08	12
+333	val_333	2008-04-08	12
+335	val_335	2008-04-08	12
+336	val_336	2008-04-08	12
+338	val_338	2008-04-08	12
+339	val_339	2008-04-08	12
+341	val_341	2008-04-08	12
+342	val_342	2008-04-08	12
+342	val_342	2008-04-08	12
+344	val_344	2008-04-08	12
+344	val_344	2008-04-08	12
+345	val_345	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+351	val_351	2008-04-08	12
+353	val_353	2008-04-08	12
+353	val_353	2008-04-08	12
+356	val_356	2008-04-08	12
+360	val_360	2008-04-08	12
+362	val_362	2008-04-08	12
+364	val_364	2008-04-08	12
+365	val_365	2008-04-08	12
+366	val_366	2008-04-08	12
+367	val_367	2008-04-08	12
+367	val_367	2008-04-08	12
+368	val_368	2008-04-08	12
+369	val_369	2008-04-08	12
+369	val_369	2008-04-08	12
+369	val_369	2008-04-08	12
+373	val_373	2008-04-08	12
+374	val_374	2008-04-08	12
+375	val_375	2008-04-08	12
+377	val_377	2008-04-08	12
+378	val_378	2008-04-08	12
+379	val_379	2008-04-08	12
+382	val_382	2008-04-08	12
+382	val_382	2008-04-08	12
+384	val_384	2008-04-08	12
+384	val_384	2008-04-08	12
+384	val_384	2008-04-08	12
+386	val_386	2008-04-08	12
+389	val_389	2008-04-08	12
+392	val_392	2008-04-08	12
+393	val_393	2008-04-08	12
+394	val_394	2008-04-08	12
+395	val_395	2008-04-08	12
+395	val_395	2008-04-08	12
+396	val_396	2008-04-08	12
+396	val_396	2008-04-08	12
+396	val_396	2008-04-08	12
+397	val_397	2008-04-08	12
+397	val_397	2008-04-08	12
+399	val_399	2008-04-08	12
+399	val_399	2008-04-08	12
+400	val_400	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+402	val_402	2008-04-08	12
+403	val_403	2008-04-08	12
+403	val_403	2008-04-08	12
+403	val_403	2008-04-08	12
+404	val_404	2008-04-08	12
+404	val_404	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+407	val_407	2008-04-08	12
+409	val_409	2008-04-08	12
+409	val_409	2008-04-08	12
+409	val_409	2008-04-08	12
+411	val_411	2008-04-08	12
+413	val_413	2008-04-08	12
+413	val_413	2008-04-08	12
+414	val_414	2008-04-08	12
+414	val_414	2008-04-08	12
+417	val_417	2008-04-08	12
+417	val_417	2008-04-08	12
+417	val_417	2008-04-08	12
+418	val_418	2008-04-08	12
+419	val_419	2008-04-08	12
+421	val_421	2008-04-08	12
+424	val_424	2008-04-08	12
+424	val_424	2008-04-08	12
+427	val_427	2008-04-08	12
+429	val_429	2008-04-08	12
+429	val_429	2008-04-08	12
+430	val_430	2008-04-08	12
+430	val_430	2008-04-08	12
+430	val_430	2008-04-08	12
+431	val_431	2008-04-08	12
+431	val_431	2008-04-08	12
+431	val_431	2008-04-08	12
+432	val_432	2008-04-08	12
+435	val_435	2008-04-08	12
+436	val_436	2008-04-08	12
+437	val_437	2008-04-08	12
+438	val_438	2008-04-08	12
+438	val_438	2008-04-08	12
+438	val_438	2008-04-08	12
+439	val_439	2008-04-08	12
+439	val_439	2008-04-08	12
+443	val_443	2008-04-08	12
+444	val_444	2008-04-08	12
+446	val_446	2008-04-08	12
+448	val_448	2008-04-08	12
+449	val_449	2008-04-08	12
+452	val_452	2008-04-08	12
+453	val_453	2008-04-08	12
+454	val_454	2008-04-08	12
+454	val_454	2008-04-08	12
+454	val_454	2008-04-08	12
+455	val_455	2008-04-08	12
+457	val_457	2008-04-08	12
+458	val_458	2008-04-08	12
+458	val_458	2008-04-08	12
+459	val_459	2008-04-08	12
+459	val_459	2008-04-08	12
+460	val_460	2008-04-08	12
+462	val_462	2008-04-08	12
+462	val_462	2008-04-08	12
+463	val_463	2008-04-08	12
+463	val_463	2008-04-08	12
+466	val_466	2008-04-08	12
+466	val_466	2008-04-08	12
+466	val_466	2008-04-08	12
+467	val_467	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+470	val_470	2008-04-08	12
+472	val_472	2008-04-08	12
+475	val_475	2008-04-08	12
+477	val_477	2008-04-08	12
+478	val_478	2008-04-08	12
+478	val_478	2008-04-08	12
+479	val_479	2008-04-08	12
+480	val_480	2008-04-08	12
+480	val_480	2008-04-08	12
+480	val_480	2008-04-08	12
+481	val_481	2008-04-08	12
+482	val_482	2008-04-08	12
+483	val_483	2008-04-08	12
+484	val_484	2008-04-08	12
+485	val_485	2008-04-08	12
+487	val_487	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+490	val_490	2008-04-08	12
+491	val_491	2008-04-08	12
+492	val_492	2008-04-08	12
+492	val_492	2008-04-08	12
+493	val_493	2008-04-08	12
+494	val_494	2008-04-08	12
+495	val_495	2008-04-08	12
+496	val_496	2008-04-08	12
+497	val_497	2008-04-08	12
+498	val_498	2008-04-08	12
+498	val_498	2008-04-08	12
+498	val_498	2008-04-08	12
+0	val_0	2008-12-31	11
+0	val_0	2008-12-31	11
+0	val_0	2008-12-31	11
+2	val_2	2008-12-31	11
+4	val_4	2008-12-31	11
+5	val_5	2008-12-31	11
+5	val_5	2008-12-31	11
+5	val_5	2008-12-31	11
+8	val_8	2008-12-31	11
+9	val_9	2008-12-31	11
+10	val_10	2008-12-31	11
+11	val_11	2008-12-31	11
+12	val_12	2008-12-31	11
+12	val_12	2008-12-31	11
+15	val_15	2008-12-31	11
+15	val_15	2008-12-31	11
+17	val_17	2008-12-31	11
+18	val_18	2008-12-31	11
+18	val_18	2008-12-31	11
+19	val_19	2008-12-31	11
+20	val_20	2008-12-31	11
+24	val_24	2008-12-31	11
+24	val_24	2008-12-31	11
+26	val_26	2008-12-31	11
+26	val_26	2008-12-31	11
+27	val_27	2008-12-31	11
+28	val_28	2008-12-31	11
+30	val_30	2008-12-31	11
+33	val_33	2008-12-31	11
+34	val_34	2008-12-31	11
+35	val_35	2008-12-31	11
+35	val_35	2008-12-31	11
+35	val_35	2008-12-31	11
+37	val_37	2008-12-31	11
+37	val_37	2008-12-31	11
+41	val_41	2008-12-31	11
+42	val_42	2008-12-31	11
+42	val_42	2008-12-31	11
+43	val_43	2008-12-31	11
+44	val_44	2008-12-31	11
+47	val_47	2008-12-31	11
+51	val_51	2008-12-31	11
+51	val_51	2008-12-31	11
+53	val_53	2008-12-31	11
+54	val_54	2008-12-31	11
+57	val_57	2008-12-31	11
+58	val_58	2008-12-31	11
+58	val_58	2008-12-31	11
+64	val_64	2008-12-31	11
+65	val_65	2008-12-31	11
+66	val_66	2008-12-31	11
+67	val_67	2008-12-31	11
+67	val_67	2008-12-31	11
+69	val_69	2008-12-31	11
+70	val_70	2008-12-31	11
+70	val_70	2008-12-31	11
+70	val_70	2008-12-31	11
+72	val_72	2008-12-31	11
+72	val_72	2008-12-31	11
+74	val_74	2008-12-31	11
+76	val_76	2008-12-31	11
+76	val_76	2008-12-31	11
+77	val_77	2008-12-31	11
+78	val_78	2008-12-31	11
+80	val_80	2008-12-31	11
+82	val_82	2008-12-31	11
+83	val_83	2008-12-31	11
+83	val_83	2008-12-31	11
+84	val_84	2008-12-31	11
+84	val_84	2008-12-31	11
+85	val_85	2008-12-31	11
+86	val_86	2008-12-31	11
+87	val_87	2008-12-31	11
+90	val_90	2008-12-31	11
+90	val_90	2008-12-31	11
+90	val_90	2008-12-31	11
+92	val_92	2008-12-31	11
+95	val_95	2008-12-31	11
+95	val_95	2008-12-31	11
+96	val_96	2008-12-31	11
+97	val_97	2008-12-31	11
+97	val_97	2008-12-31	11
+98	val_98	2008-12-31	11
+98	val_98	2008-12-31	11
+100	val_100	2008-12-31	11
+100	val_100	2008-12-31	11
+103	val_103	2008-12-31	11
+103	val_103	2008-12-31	11
+104	val_104	2008-12-31	11
+104	val_104	2008-12-31	11
+105	val_105	2008-12-31	11
+111	val_111	2008-12-31	11
+113	val_113	2008-12-31	11
+113	val_113	2008-12-31	11
+114	val_114	2008-12-31	11
+116	val_116	2008-12-31	11
+118	val_118	2008-12-31	11
+118	val_118	2008-12-31	11
+119	val_119	2008-12-31	11
+119	val_119	2008-12-31	11
+119	val_119	2008-12-31	11
+120	val_120	2008-12-31	11
+120	val_120	2008-12-31	11
+125	val_125	2008-12-31	11
+125	val_125	2008-12-31	11
+126	val_126	2008-12-31	11
+128	val_128	2008-12-31	11
+128	val_128	2008-12-31	11
+128	val_128	2008-12-31	11
+129	val_129	2008-12-31	11
+129	val_129	2008-12-31	11
+131	val_131	2008-12-31	11
+133	val_133	2008-12-31	11
+134	val_134	2008-12-31	11
+134	val_134	2008-12-31	11
+136	val_136	2008-12-31	11
+137	val_137	2008-12-31	11
+137	val_137	2008-12-31	11
+138	val_138	2008-12-31	11
+138	val_138	2008-12-31	11
+138	val_138	2008-12-31	11
+138	val_138	2008-12-31	11
+143	val_143	2008-12-31	11
+145	val_145	2008-12-31	11
+146	val_146	2008-12-31	11
+146	val_146	2008-12-31	11
+149	val_149	2008-12-31	11
+149	val_149	2008-12-31	11
+150	val_150	2008-12-31	11
+152	val_152	2008-12-31	11
+152	val_152	2008-12-31	11
+153	val_153	2008-12-31	11
+155	val_155	2008-12-31	11
+156	val_156	2008-12-31	11
+157	val_157	2008-12-31	11
+158	val_158	2008-12-31	11
+160	val_160	2008-12-31	11
+162	val_162	2008-12-31	11
+163	val_163	2008-12-31	11
+164	val_164	2008-12-31	11
+164	val_164	2008-12-31	11
+165	val_165	2008-12-31	11
+165	val_165	2008-12-31	11
+166	val_166	2008-12-31	11
+167	val_167	2008-12-31	11
+167	val_167	2008-12-31	11
+167	val_167	2008-12-31	11
+168	val_168	2008-12-31	11
+169	val_169	2008-12-31	11
+169	val_169	2008-12-31	11
+169	val_169	2008-12-31	11
+169	val_169	2008-12-31	11
+170	val_170	2008-12-31	11
+172	val_172	2008-12-31	11
+172	val_172	2008-12-31	11
+174	val_174	2008-12-31	11
+174	val_174	2008-12-31	11
+175	val_175	2008-12-31	11
+175	val_175	2008-12-31	11
+176	val_176	2008-12-31	11
+176	val_176	2008-12-31	11
+177	val_177	2008-12-31	11
+178	val_178	2008-12-31	11
+179	val_179	2008-12-31	11
+179	val_179	2008-12-31	11
+180	val_180	2008-12-31	11
+181	val_181	2008-12-31	11
+183	val_183	2008-12-31	11
+186	val_186	2008-12-31	11
+187	val_187	2008-12-31	11
+187	val_187	2008-12-31	11
+187	val_187	2008-12-31	11
+189	val_189	2008-12-31	11
+190	val_190	2008-12-31	11
+191	val_191	2008-12-31	11
+191	val_191	2008-12-31	11
+192	val_192	2008-12-31	11
+193	val_193	2008-12-31	11
+193	val_193	2008-12-31	11
+193	val_193	2008-12-31	11
+194	val_194	2008-12-31	11
+195	val_195	2008-12-31	11
+195	val_195	2008-12-31	11
+196	val_196	2008-12-31	11
+197	val_197	2008-12-31	11
+197	val_197	2008-12-31	11
+199	val_199	2008-12-31	11
+199	val_199	2008-12-31	11
+199	val_199	2008-12-31	11
+200	val_200	2008-12-31	11
+200	val_200	2008-12-31	11
+201	val_201	2008-12-31	11
+202	val_202	2008-12-31	11
+203	val_203	2008-12-31	11
+203	val_203	2008-12-31	11
+205	val_205	2008-12-31	11
+205	val_205	2008-12-31	11
+207	val_207	2008-12-31	11
+207	val_207	2008-12-31	11
+208	val_208	2008-12-31	11
+208	val_208	2008-12-31	11
+208	val_208	2008-12-31	11
+209	val_209	2008-12-31	11
+209	val_209	2008-12-31	11
+213	val_213	2008-12-31	11
+213	val_213	2008-12-31	11
+214	val_214	2008-12-31	11
+216	val_216	2008-12-31	11
+216	val_216	2008-12-31	11
+217	val_217	2008-12-31	11
+217	val_217	2008-12-31	11
+218	val_218	2008-12-31	11
+219	val_219	2008-12-31	11
+219	val_219	2008-12-31	11
+221	val_221	2008-12-31	11
+221	val_221	2008-12-31	11
+222	val_222	2008-12-31	11
+223	val_223	2008-12-31	11
+223	val_223	2008-12-31	11
+224	val_224	2008-12-31	11
+224	val_224	2008-12-31	11
+226	val_226	2008-12-31	11
+228	val_228	2008-12-31	11
+229	val_229	2008-12-31	11
+229	val_229	2008-12-31	11
+230	val_230	2008-12-31	11
+230	val_230	2008-12-31	11
+230	val_230	2008-12-31	11
+230	val_230	2008-12-31	11
+230	val_230	2008-12-31	11
+233	val_233	2008-12-31	11
+233	val_233	2008-12-31	11
+235	val_235	2008-12-31	11
+237	val_237	2008-12-31	11
+237	val_237	2008-12-31	11
+238	val_238	2008-12-31	11
+238	val_238	2008-12-31	11
+239	val_239	2008-12-31	11
+239	val_239	2008-12-31	11
+241	val_241	2008-12-31	11
+242	val_242	2008-12-31	11
+242	val_242	2008-12-31	11
+244	val_244	2008-12-31	11
+247	val_247	2008-12-31	11
+248	val_248	2008-12-31	11
+249	val_249	2008-12-31	11
+252	val_252	2008-12-31	11
+255	val_255	2008-12-31	11
+255	val_255	2008-12-31	11
+256	val_256	2008-12-31	11
+256	val_256	2008-12-31	11
+257	val_257	2008-12-31	11
+258	val_258	2008-12-31	11
+260	val_260	2008-12-31	11
+262	val_262	2008-12-31	11
+263	val_263	2008-12-31	11
+265	val_265	2008-12-31	11
+265	val_265	2008-12-31	11
+266	val_266	2008-12-31	11
+272	val_272	2008-12-31	11
+272	val_272	2008-12-31	11
+273	val_273	2008-12-31	11
+273	val_273	2008-12-31	11
+273	val_273	2008-12-31	11
+274	val_274	2008-12-31	11
+275	val_275	2008-12-31	11
+277	val_277	2008-12-31	11
+277	val_277	2008-12-31	11
+277	val_277	2008-12-31	11
+277	val_277	2008-12-31	11
+278	val_278	2008-12-31	11
+278	val_278	2008-12-31	11
+280	val_280	2008-12-31	11
+280	val_280	2008-12-31	11
+281	val_281	2008-12-31	11
+281	val_281	2008-12-31	11
+282	val_282	2008-12-31	11
+282	val_282	2008-12-31	11
+283	val_283	2008-12-31	11
+284	val_284	2008-12-31	11
+285	val_285	2008-12-31	11
+286	val_286	2008-12-31	11
+287	val_287	2008-12-31	11
+288	val_288	2008-12-31	11
+288	val_288	2008-12-31	11
+289	val_289	2008-12-31	11
+291	val_291	2008-12-31	11
+292	val_292	2008-12-31	11
+296	val_296	2008-12-31	11
+298	val_298	2008-12-31	11
+298	val_298	2008-12-31	11
+298	val_298	2008-12-31	11
+302	val_302	2008-12-31	11
+305	val_305	2008-12-31	11
+306	val_306	2008-12-31	11
+307	val_307	2008-12-31	11
+307	val_307	2008-12-31	11
+308	val_308	2008-12-31	11
+309	val_309	2008-12-31	11
+309	val_309	2008-12-31	11
+310	val_310	2008-12-31	11
+311	val_311	2008-12-31	11
+311	val_311	2008-12-31	11
+311	val_311	2008-12-31	11
+315	val_315	2008-12-31	11
+316	val_316	2008-12-31	11
+316	val_316	2008-12-31	11
+316	val_316	2008-12-31	11
+317	val_317	2008-12-31	11
+317	val_317	2008-12-31	11
+318	val_318	2008-12-31	11
+318	val_318	2008-12-31	11
+318	val_318	2008-12-31	11
+321	val_321	2008-12-31	11
+321	val_321	2008-12-31	11
+322	val_322	2008-12-31	11
+322	val_322	2008-12-31	11
+323	val_323	2008-12-31	11
+325	val_325	2008-12-31	11
+325	val_325	2008-12-31	11
+327	val_327	2008-12-31	11
+327	val_327	2008-12-31	11
+327	val_327	2008-12-31	11
+331	val_331	2008-12-31	11
+331	val_331	2008-12-31	11
+332	val_332	2008-12-31	11
+333	val_333	2008-12-31	11
+333	val_333	2008-12-31	11
+335	val_335	2008-12-31	11
+336	val_336	2008-12-31	11
+338	val_338	2008-12-31	11
+339	val_339	2008-12-31	11
+341	val_341	2008-12-31	11
+342	val_342	2008-12-31	11
+342	val_342	2008-12-31	11
+344	val_344	2008-12-31	11
+344	val_344	2008-12-31	11
+345	val_345	2008-12-31	11
+348	val_348	2008-12-31	11
+348	val_348	2008-12-31	11
+348	val_348	2008-12-31	11
+348	val_348	2008-12-31	11
+348	val_348	2008-12-31	11
+351	val_351	2008-12-31	11
+353	val_353	2008-12-31	11
+353	val_353	2008-12-31	11
+356	val_356	2008-12-31	11
+360	val_360	2008-12-31	11
+362	val_362	2008-12-31	11
+364	val_364	2008-12-31	11
+365	val_365	2008-12-31	11
+366	val_366	2008-12-31	11
+367	val_367	2008-12-31	11
+367	val_367	2008-12-31	11
+368	val_368	2008-12-31	11
+369	val_369	2008-12-31	11
+369	val_369	2008-12-31	11
+369	val_369	2008-12-31	11
+373	val_373	2008-12-31	11
+374	val_374	2008-12-31	11
+375	val_375	2008-12-31	11
+377	val_377	2008-12-31	11
+378	val_378	2008-12-31	11
+379	val_379	2008-12-31	11
+382	val_382	2008-12-31	11
+382	val_382	2008-12-31	11
+384	val_384	2008-12-31	11
+384	val_384	2008-12-31	11
+384	val_384	2008-12-31	11
+386	val_386	2008-12-31	11
+389	val_389	2008-12-31	11
+392	val_392	2008-12-31	11
+393	val_393	2008-12-31	11
+394	val_394	2008-12-31	11
+395	val_395	2008-12-31	11
+395	val_395	2008-12-31	11
+396	val_396	2008-12-31	11
+396	val_396	2008-12-31	11
+396	val_396	2008-12-31	11
+397	val_397	2008-12-31	11
+397	val_397	2008-12-31	11
+399	val_399	2008-12-31	11
+399	val_399	2008-12-31	11
+400	val_400	2008-12-31	11
+401	val_401	2008-12-31	11
+401	val_401	2008-12-31	11
+401	val_401	2008-12-31	11
+401	val_401	2008-12-31	11
+401	val_401	2008-12-31	11
+402	val_402	2008-12-31	11
+403	val_403	2008-12-31	11
+403	val_403	2008-12-31	11
+403	val_403	2008-12-31	11
+404	val_404	2008-12-31	11
+404	val_404	2008-12-31	11
+406	val_406	2008-12-31	11
+406	val_406	2008-12-31	11
+406	val_406	2008-12-31	11
+406	val_406	2008-12-31	11
+407	val_407	2008-12-31	11
+409	val_409	2008-12-31	11
+409	val_409	2008-12-31	11
+409	val_409	2008-12-31	11
+411	val_411	2008-12-31	11
+413	val_413	2008-12-31	11
+413	val_413	2008-12-31	11
+414	val_414	2008-12-31	11
+414	val_414	2008-12-31	11
+417	val_417	2008-12-31	11
+417	val_417	2008-12-31	11
+417	val_417	2008-12-31	11
+418	val_418	2008-12-31	11
+419	val_419	2008-12-31	11
+421	val_421	2008-12-31	11
+424	val_424	2008-12-31	11
+424	val_424	2008-12-31	11
+427	val_427	2008-12-31	11
+429	val_429	2008-12-31	11
+429	val_429	2008-12-31	11
+430	val_430	2008-12-31	11
+430	val_430	2008-12-31	11
+430	val_430	2008-12-31	11
+431	val_431	2008-12-31	11
+431	val_431	2008-12-31	11
+431	val_431	2008-12-31	11
+432	val_432	2008-12-31	11
+435	val_435	2008-12-31	11
+436	val_436	2008-12-31	11
+437	val_437	2008-12-31	11
+438	val_438	2008-12-31	11
+438	val_438	2008-12-31	11
+438	val_438	2008-12-31	11
+439	val_439	2008-12-31	11
+439	val_439	2008-12-31	11
+443	val_443	2008-12-31	11
+444	val_444	2008-12-31	11
+446	val_446	2008-12-31	11
+448	val_448	2008-12-31	11
+449	val_449	2008-12-31	11
+452	val_452	2008-12-31	11
+453	val_453	2008-12-31	11
+454	val_454	2008-12-31	11
+454	val_454	2008-12-31	11
+454	val_454	2008-12-31	11
+455	val_455	2008-12-31	11
+457	val_457	2008-12-31	11
+458	val_458	2008-12-31	11
+458	val_458	2008-12-31	11
+459	val_459	2008-12-31	11
+459	val_459	2008-12-31	11
+460	val_460	2008-12-31	11
+462	val_462	2008-12-31	11
+462	val_462	2008-12-31	11
+463	val_463	2008-12-31	11
+463	val_463	2008-12-31	11
+466	val_466	2008-12-31	11
+466	val_466	2008-12-31	11
+466	val_466	2008-12-31	11
+467	val_467	2008-12-31	11
+468	val_468	2008-12-31	11
+468	val_468	2008-12-31	11
+468	val_468	2008-12-31	11
+468	val_468	2008-12-31	11
+469	val_469	2008-12-31	11
+469	val_469	2008-12-31	11
+469	val_469	2008-12-31	11
+469	val_469	2008-12-31	11
+469	val_469	2008-12-31	11
+470	val_470	2008-12-31	11
+472	val_472	2008-12-31	11
+475	val_475	2008-12-31	11
+477	val_477	2008-12-31	11
+478	val_478	2008-12-31	11
+478	val_478	2008-12-31	11
+479	val_479	2008-12-31	11
+480	val_480	2008-12-31	11
+480	val_480	2008-12-31	11
+480	val_480	2008-12-31	11
+481	val_481	2008-12-31	11
+482	val_482	2008-12-31	11
+483	val_483	2008-12-31	11
+484	val_484	2008-12-31	11
+485	val_485	2008-12-31	11
+487	val_487	2008-12-31	11
+489	val_489	2008-12-31	11
+489	val_489	2008-12-31	11
+489	val_489	2008-12-31	11
+489	val_489	2008-12-31	11
+490	val_490	2008-12-31	11
+491	val_491	2008-12-31	11
+492	val_492	2008-12-31	11
+492	val_492	2008-12-31	11
+493	val_493	2008-12-31	11
+494	val_494	2008-12-31	11
+495	val_495	2008-12-31	11
+496	val_496	2008-12-31	11
+497	val_497	2008-12-31	11
+498	val_498	2008-12-31	11
+498	val_498	2008-12-31	11
+498	val_498	2008-12-31	11
+0	val_0	2008-12-31	12
+0	val_0	2008-12-31	12
+0	val_0	2008-12-31	12
+2	val_2	2008-12-31	12
+4	val_4	2008-12-31	12
+5	val_5	2008-12-31	12
+5	val_5	2008-12-31	12
+5	val_5	2008-12-31	12
+8	val_8	2008-12-31	12
+9	val_9	2008-12-31	12
+10	val_10	2008-12-31	12
+11	val_11	2008-12-31	12
+12	val_12	2008-12-31	12
+12	val_12	2008-12-31	12
+15	val_15	2008-12-31	12
+15	val_15	2008-12-31	12
+17	val_17	2008-12-31	12
+18	val_18	2008-12-31	12
+18	val_18	2008-12-31	12
+19	val_19	2008-12-31	12
+20	val_20	2008-12-31	12
+24	val_24	2008-12-31	12
+24	val_24	2008-12-31	12
+26	val_26	2008-12-31	12
+26	val_26	2008-12-31	12
+27	val_27	2008-12-31	12
+28	val_28	2008-12-31	12
+30	val_30	2008-12-31	12
+33	val_33	2008-12-31	12
+34	val_34	2008-12-31	12
+35	val_35	2008-12-31	12
+35	val_35	2008-12-31	12
+35	val_35	2008-12-31	12
+37	val_37	2008-12-31	12
+37	val_37	2008-12-31	12
+41	val_41	2008-12-31	12
+42	val_42	2008-12-31	12
+42	val_42	2008-12-31	12
+43	val_43	2008-12-31	12
+44	val_44	2008-12-31	12
+47	val_47	2008-12-31	12
+51	val_51	2008-12-31	12
+51	val_51	2008-12-31	12
+53	val_53	2008-12-31	12
+54	val_54	2008-12-31	12
+57	val_57	2008-12-31	12
+58	val_58	2008-12-31	12
+58	val_58	2008-12-31	12
+64	val_64	2008-12-31	12
+65	val_65	2008-12-31	12
+66	val_66	2008-12-31	12
+67	val_67	2008-12-31	12
+67	val_67	2008-12-31	12
+69	val_69	2008-12-31	12
+70	val_70	2008-12-31	12
+70	val_70	2008-12-31	12
+70	val_70	2008-12-31	12
+72	val_72	2008-12-31	12
+72	val_72	2008-12-31	12
+74	val_74	2008-12-31	12
+76	val_76	2008-12-31	12
+76	val_76	2008-12-31	12
+77	val_77	2008-12-31	12
+78	val_78	2008-12-31	12
+80	val_80	2008-12-31	12
+82	val_82	2008-12-31	12
+83	val_83	2008-12-31	12
+83	val_83	2008-12-31	12
+84	val_84	2008-12-31	12
+84	val_84	2008-12-31	12
+85	val_85	2008-12-31	12
+86	val_86	2008-12-31	12
+87	val_87	2008-12-31	12
+90	val_90	2008-12-31	12
+90	val_90	2008-12-31	12
+90	val_90	2008-12-31	12
+92	val_92	2008-12-31	12
+95	val_95	2008-12-31	12
+95	val_95	2008-12-31	12
+96	val_96	2008-12-31	12
+97	val_97	2008-12-31	12
+97	val_97	2008-12-31	12
+98	val_98	2008-12-31	12
+98	val_98	2008-12-31	12
+100	val_100	2008-12-31	12
+100	val_100	2008-12-31	12
+103	val_103	2008-12-31	12
+103	val_103	2008-12-31	12
+104	val_104	2008-12-31	12
+104	val_104	2008-12-31	12
+105	val_105	2008-12-31	12
+111	val_111	2008-12-31	12
+113	val_113	2008-12-31	12
+113	val_113	2008-12-31	12
+114	val_114	2008-12-31	12
+116	val_116	2008-12-31	12
+118	val_118	2008-12-31	12
+118	val_118	2008-12-31	12
+119	val_119	2008-12-31	12
+119	val_119	2008-12-31	12
+119	val_119	2008-12-31	12
+120	val_120	2008-12-31	12
+120	val_120	2008-12-31	12
+125	val_125	2008-12-31	12
+125	val_125	2008-12-31	12
+126	val_126	2008-12-31	12
+128	val_128	2008-12-31	12
+128	val_128	2008-12-31	12
+128	val_128	2008-12-31	12
+129	val_129	2008-12-31	12
+129	val_129	2008-12-31	12
+131	val_131	2008-12-31	12
+133	val_133	2008-12-31	12
+134	val_134	2008-12-31	12
+134	val_134	2008-12-31	12
+136	val_136	2008-12-31	12
+137	val_137	2008-12-31	12
+137	val_137	2008-12-31	12
+138	val_138	2008-12-31	12
+138	val_138	2008-12-31	12
+138	val_138	2008-12-31	12
+138	val_138	2008-12-31	12
+143	val_143	2008-12-31	12
+145	val_145	2008-12-31	12
+146	val_146	2008-12-31	12
+146	val_146	2008-12-31	12
+149	val_149	2008-12-31	12
+149	val_149	2008-12-31	12
+150	val_150	2008-12-31	12
+152	val_152	2008-12-31	12
+152	val_152	2008-12-31	12
+153	val_153	2008-12-31	12
+155	val_155	2008-12-31	12
+156	val_156	2008-12-31	12
+157	val_157	2008-12-31	12
+158	val_158	2008-12-31	12
+160	val_160	2008-12-31	12
+162	val_162	2008-12-31	12
+163	val_163	2008-12-31	12
+164	val_164	2008-12-31	12
+164	val_164	2008-12-31	12
+165	val_165	2008-12-31	12
+165	val_165	2008-12-31	12
+166	val_166	2008-12-31	12
+167	val_167	2008-12-31	12
+167	val_167	2008-12-31	12
+167	val_167	2008-12-31	12
+168	val_168	2008-12-31	12
+169	val_169	2008-12-31	12
+169	val_169	2008-12-31	12
+169	val_169	2008-12-31	12
+169	val_169	2008-12-31	12
+170	val_170	2008-12-31	12
+172	val_172	2008-12-31	12
+172	val_172	2008-12-31	12
+174	val_174	2008-12-31	12
+174	val_174	2008-12-31	12
+175	val_175	2008-12-31	12
+175	val_175	2008-12-31	12
+176	val_176	2008-12-31	12
+176	val_176	2008-12-31	12
+177	val_177	2008-12-31	12
+178	val_178	2008-12-31	12
+179	val_179	2008-12-31	12
+179	val_179	2008-12-31	12
+180	val_180	2008-12-31	12
+181	val_181	2008-12-31	12
+183	val_183	2008-12-31	12
+186	val_186	2008-12-31	12
+187	val_187	2008-12-31	12
+187	val_187	2008-12-31	12
+187	val_187	2008-12-31	12
+189	val_189	2008-12-31	12
+190	val_190	2008-12-31	12
+191	val_191	2008-12-31	12
+191	val_191	2008-12-31	12
+192	val_192	2008-12-31	12
+193	val_193	2008-12-31	12
+193	val_193	2008-12-31	12
+193	val_193	2008-12-31	12
+194	val_194	2008-12-31	12
+195	val_195	2008-12-31	12
+195	val_195	2008-12-31	12
+196	val_196	2008-12-31	12
+197	val_197	2008-12-31	12
+197	val_197	2008-12-31	12
+199	val_199	2008-12-31	12
+199	val_199	2008-12-31	12
+199	val_199	2008-12-31	12
+200	val_200	2008-12-31	12
+200	val_200	2008-12-31	12
+201	val_201	2008-12-31	12
+202	val_202	2008-12-31	12
+203	val_203	2008-12-31	12
+203	val_203	2008-12-31	12
+205	val_205	2008-12-31	12
+205	val_205	2008-12-31	12
+207	val_207	2008-12-31	12
+207	val_207	2008-12-31	12
+208	val_208	2008-12-31	12
+208	val_208	2008-12-31	12
+208	val_208	2008-12-31	12
+209	val_209	2008-12-31	12
+209	val_209	2008-12-31	12
+213	val_213	2008-12-31	12
+213	val_213	2008-12-31	12
+214	val_214	2008-12-31	12
+216	val_216	2008-12-31	12
+216	val_216	2008-12-31	12
+217	val_217	2008-12-31	12
+217	val_217	2008-12-31	12
+218	val_218	2008-12-31	12
+219	val_219	2008-12-31	12
+219	val_219	2008-12-31	12
+221	val_221	2008-12-31	12
+221	val_221	2008-12-31	12
+222	val_222	2008-12-31	12
+223	val_223	2008-12-31	12
+223	val_223	2008-12-31	12
+224	val_224	2008-12-31	12
+224	val_224	2008-12-31	12
+226	val_226	2008-12-31	12
+228	val_228	2008-12-31	12
+229	val_229	2008-12-31	12
+229	val_229	2008-12-31	12
+230	val_230	2008-12-31	12
+230	val_230	2008-12-31	12
+230	val_230	2008-12-31	12
+230	val_230	2008-12-31	12
+230	val_230	2008-12-31	12
+233	val_233	2008-12-31	12
+233	val_233	2008-12-31	12
+235	val_235	2008-12-31	12
+237	val_237	2008-12-31	12
+237	val_237	2008-12-31	12
+238	val_238	2008-12-31	12
+238	val_238	2008-12-31	12
+239	val_239	2008-12-31	12
+239	val_239	2008-12-31	12
+241	val_241	2008-12-31	12
+242	val_242	2008-12-31	12
+242	val_242	2008-12-31	12
+244	val_244	2008-12-31	12
+247	val_247	2008-12-31	12
+248	val_248	2008-12-31	12
+249	val_249	2008-12-31	12
+252	val_252	2008-12-31	12
+255	val_255	2008-12-31	12
+255	val_255	2008-12-31	12
+256	val_256	2008-12-31	12
+256	val_256	2008-12-31	12
+257	val_257	2008-12-31	12
+258	val_258	2008-12-31	12
+260	val_260	2008-12-31	12
+262	val_262	2008-12-31	12
+263	val_263	2008-12-31	12
+265	val_265	2008-12-31	12
+265	val_265	2008-12-31	12
+266	val_266	2008-12-31	12
+272	val_272	2008-12-31	12
+272	val_272	2008-12-31	12
+273	val_273	2008-12-31	12
+273	val_273	2008-12-31	12
+273	val_273	2008-12-31	12
+274	val_274	2008-12-31	12
+275	val_275	2008-12-31	12
+277	val_277	2008-12-31	12
+277	val_277	2008-12-31	12
+277	val_277	2008-12-31	12
+277	val_277	2008-12-31	12
+278	val_278	2008-12-31	12
+278	val_278	2008-12-31	12
+280	val_280	2008-12-31	12
+280	val_280	2008-12-31	12
+281	val_281	2008-12-31	12
+281	val_281	2008-12-31	12
+282	val_282	2008-12-31	12
+282	val_282	2008-12-31	12
+283	val_283	2008-12-31	12
+284	val_284	2008-12-31	12
+285	val_285	2008-12-31	12
+286	val_286	2008-12-31	12
+287	val_287	2008-12-31	12
+288	val_288	2008-12-31	12
+288	val_288	2008-12-31	12
+289	val_289	2008-12-31	12
+291	val_291	2008-12-31	12
+292	val_292	2008-12-31	12
+296	val_296	2008-12-31	12
+298	val_298	2008-12-31	12
+298	val_298	2008-12-31	12
+298	val_298	2008-12-31	12
+302	val_302	2008-12-31	12
+305	val_305	2008-12-31	12
+306	val_306	2008-12-31	12
+307	val_307	2008-12-31	12
+307	val_307	2008-12-31	12
+308	val_308	2008-12-31	12
+309	val_309	2008-12-31	12
+309	val_309	2008-12-31	12
+310	val_310	2008-12-31	12
+311	val_311	2008-12-31	12
+311	val_311	2008-12-31	12
+311	val_311	2008-12-31	12
+315	val_315	2008-12-31	12
+316	val_316	2008-12-31	12
+316	val_316	2008-12-31	12
+316	val_316	2008-12-31	12
+317	val_317	2008-12-31	12
+317	val_317	2008-12-31	12
+318	val_318	2008-12-31	12
+318	val_318	2008-12-31	12
+318	val_318	2008-12-31	12
+321	val_321	2008-12-31	12
+321	val_321	2008-12-31	12
+322	val_322	2008-12-31	12
+322	val_322	2008-12-31	12
+323	val_323	2008-12-31	12
+325	val_325	2008-12-31	12
+325	val_325	2008-12-31	12
+327	val_327	2008-12-31	12
+327	val_327	2008-12-31	12
+327	val_327	2008-12-31	12
+331	val_331	2008-12-31	12
+331	val_331	2008-12-31	12
+332	val_332	2008-12-31	12
+333	val_333	2008-12-31	12
+333	val_333	2008-12-31	12
+335	val_335	2008-12-31	12
+336	val_336	2008-12-31	12
+338	val_338	2008-12-31	12
+339	val_339	2008-12-31	12
+341	val_341	2008-12-31	12
+342	val_342	2008-12-31	12
+342	val_342	2008-12-31	12
+344	val_344	2008-12-31	12
+344	val_344	2008-12-31	12
+345	val_345	2008-12-31	12
+348	val_348	2008-12-31	12
+348	val_348	2008-12-31	12
+348	val_348	2008-12-31	12
+348	val_348	2008-12-31	12
+348	val_348	2008-12-31	12
+351	val_351	2008-12-31	12
+353	val_353	2008-12-31	12
+353	val_353	2008-12-31	12
+356	val_356	2008-12-31	12
+360	val_360	2008-12-31	12
+362	val_362	2008-12-31	12
+364	val_364	2008-12-31	12
+365	val_365	2008-12-31	12
+366	val_366	2008-12-31	12
+367	val_367	2008-12-31	12
+367	val_367	2008-12-31	12
+368	val_368	2008-12-31	12
+369	val_369	2008-12-31	12
+369	val_369	2008-12-31	12
+369	val_369	2008-12-31	12
+373	val_373	2008-12-31	12
+374	val_374	2008-12-31	12
+375	val_375	2008-12-31	12
+377	val_377	2008-12-31	12
+378	val_378	2008-12-31	12
+379	val_379	2008-12-31	12
+382	val_382	2008-12-31	12
+382	val_382	2008-12-31	12
+384	val_384	2008-12-31	12
+384	val_384	2008-12-31	12
+384	val_384	2008-12-31	12
+386	val_386	2008-12-31	12
+389	val_389	2008-12-31	12
+392	val_392	2008-12-31	12
+393	val_393	2008-12-31	12
+394	val_394	2008-12-31	12
+395	val_395	2008-12-31	12
+395	val_395	2008-12-31	12
+396	val_396	2008-12-31	12
+396	val_396	2008-12-31	12
+396	val_396	2008-12-31	12
+397	val_397	2008-12-31	12
+397	val_397	2008-12-31	12
+399	val_399	2008-12-31	12
+399	val_399	2008-12-31	12
+400	val_400	2008-12-31	12
+401	val_401	2008-12-31	12
+401	val_401	2008-12-31	12
+401	val_401	2008-12-31	12
+401	val_401	2008-12-31	12
+401	val_401	2008-12-31	12
+402	val_402	2008-12-31	12
+403	val_403	2008-12-31	12
+403	val_403	2008-12-31	12
+403	val_403	2008-12-31	12
+404	val_404	2008-12-31	12
+404	val_404	2008-12-31	12
+406	val_406	2008-12-31	12
+406	val_406	2008-12-31	12
+406	val_406	2008-12-31	12
+406	val_406	2008-12-31	12
+407	val_407	2008-12-31	12
+409	val_409	2008-12-31	12
+409	val_409	2008-12-31	12
+409	val_409	2008-12-31	12
+411	val_411	2008-12-31	12
+413	val_413	2008-12-31	12
+413	val_413	2008-12-31	12
+414	val_414	2008-12-31	12
+414	val_414	2008-12-31	12
+417	val_417	2008-12-31	12
+417	val_417	2008-12-31	12
+417	val_417	2008-12-31	12
+418	val_418	2008-12-31	12
+419	val_419	2008-12-31	12
+421	val_421	2008-12-31	12
+424	val_424	2008-12-31	12
+424	val_424	2008-12-31	12
+427	val_427	2008-12-31	12
+429	val_429	2008-12-31	12
+429	val_429	2008-12-31	12
+430	val_430	2008-12-31	12
+430	val_430	2008-12-31	12
+430	val_430	2008-12-31	12
+431	val_431	2008-12-31	12
+431	val_431	2008-12-31	12
+431	val_431	2008-12-31	12
+432	val_432	2008-12-31	12
+435	val_435	2008-12-31	12
+436	val_436	2008-12-31	12
+437	val_437	2008-12-31	12
+438	val_438	2008-12-31	12
+438	val_438	2008-12-31	12
+438	val_438	2008-12-31	12
+439	val_439	2008-12-31	12
+439	val_439	2008-12-31	12
+443	val_443	2008-12-31	12
+444	val_444	2008-12-31	12
+446	val_446	2008-12-31	12
+448	val_448	2008-12-31	12
+449	val_449	2008-12-31	12
+452	val_452	2008-12-31	12
+453	val_453	2008-12-31	12
+454	val_454	2008-12-31	12
+454	val_454	2008-12-31	12
+454	val_454	2008-12-31	12
+455	val_455	2008-12-31	12
+457	val_457	2008-12-31	12
+458	val_458	2008-12-31	12
+458	val_458	2008-12-31	12
+459	val_459	2008-12-31	12
+459	val_459	2008-12-31	12
+460	val_460	2008-12-31	12
+462	val_462	2008-12-31	12
+462	val_462	2008-12-31	12
+463	val_463	2008-12-31	12
+463	val_463	2008-12-31	12
+466	val_466	2008-12-31	12
+466	val_466	2008-12-31	12
+466	val_466	2008-12-31	12
+467	val_467	2008-12-31	12
+468	val_468	2008-12-31	12
+468	val_468	2008-12-31	12
+468	val_468	2008-12-31	12
+468	val_468	2008-12-31	12
+469	val_469	2008-12-31	12
+469	val_469	2008-12-31	12
+469	val_469	2008-12-31	12
+469	val_469	2008-12-31	12
+469	val_469	2008-12-31	12
+470	val_470	2008-12-31	12
+472	val_472	2008-12-31	12
+475	val_475	2008-12-31	12
+477	val_477	2008-12-31	12
+478	val_478	2008-12-31	12
+478	val_478	2008-12-31	12
+479	val_479	2008-12-31	12
+480	val_480	2008-12-31	12
+480	val_480	2008-12-31	12
+480	val_480	2008-12-31	12
+481	val_481	2008-12-31	12
+482	val_482	2008-12-31	12
+483	val_483	2008-12-31	12
+484	val_484	2008-12-31	12
+485	val_485	2008-12-31	12
+487	val_487	2008-12-31	12
+489	val_489	2008-12-31	12
+489	val_489	2008-12-31	12
+489	val_489	2008-12-31	12
+489	val_489	2008-12-31	12
+490	val_490	2008-12-31	12
+491	val_491	2008-12-31	12
+492	val_492	2008-12-31	12
+492	val_492	2008-12-31	12
+493	val_493	2008-12-31	12
+494	val_494	2008-12-31	12
+495	val_495	2008-12-31	12
+496	val_496	2008-12-31	12
+497	val_497	2008-12-31	12
+498	val_498	2008-12-31	12
+498	val_498	2008-12-31	12
+498	val_498	2008-12-31	12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db b/sql/hive/src/test/resources/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db
deleted file mode 100644
index 4c0ed5d6fc5f1cc63ee415be916b84ae43d9410b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db
+++ /dev/null
@@ -1,2000 +0,0 @@
-238	val_238	2008-04-08	11
-86	val_86	2008-04-08	11
-311	val_311	2008-04-08	11
-27	val_27	2008-04-08	11
-165	val_165	2008-04-08	11
-409	val_409	2008-04-08	11
-255	val_255	2008-04-08	11
-278	val_278	2008-04-08	11
-98	val_98	2008-04-08	11
-484	val_484	2008-04-08	11
-265	val_265	2008-04-08	11
-193	val_193	2008-04-08	11
-401	val_401	2008-04-08	11
-150	val_150	2008-04-08	11
-273	val_273	2008-04-08	11
-224	val_224	2008-04-08	11
-369	val_369	2008-04-08	11
-66	val_66	2008-04-08	11
-128	val_128	2008-04-08	11
-213	val_213	2008-04-08	11
-146	val_146	2008-04-08	11
-406	val_406	2008-04-08	11
-429	val_429	2008-04-08	11
-374	val_374	2008-04-08	11
-152	val_152	2008-04-08	11
-469	val_469	2008-04-08	11
-145	val_145	2008-04-08	11
-495	val_495	2008-04-08	11
-37	val_37	2008-04-08	11
-327	val_327	2008-04-08	11
-281	val_281	2008-04-08	11
-277	val_277	2008-04-08	11
-209	val_209	2008-04-08	11
-15	val_15	2008-04-08	11
-82	val_82	2008-04-08	11
-403	val_403	2008-04-08	11
-166	val_166	2008-04-08	11
-417	val_417	2008-04-08	11
-430	val_430	2008-04-08	11
-252	val_252	2008-04-08	11
-292	val_292	2008-04-08	11
-219	val_219	2008-04-08	11
-287	val_287	2008-04-08	11
-153	val_153	2008-04-08	11
-193	val_193	2008-04-08	11
-338	val_338	2008-04-08	11
-446	val_446	2008-04-08	11
-459	val_459	2008-04-08	11
-394	val_394	2008-04-08	11
-237	val_237	2008-04-08	11
-482	val_482	2008-04-08	11
-174	val_174	2008-04-08	11
-413	val_413	2008-04-08	11
-494	val_494	2008-04-08	11
-207	val_207	2008-04-08	11
-199	val_199	2008-04-08	11
-466	val_466	2008-04-08	11
-208	val_208	2008-04-08	11
-174	val_174	2008-04-08	11
-399	val_399	2008-04-08	11
-396	val_396	2008-04-08	11
-247	val_247	2008-04-08	11
-417	val_417	2008-04-08	11
-489	val_489	2008-04-08	11
-162	val_162	2008-04-08	11
-377	val_377	2008-04-08	11
-397	val_397	2008-04-08	11
-309	val_309	2008-04-08	11
-365	val_365	2008-04-08	11
-266	val_266	2008-04-08	11
-439	val_439	2008-04-08	11
-342	val_342	2008-04-08	11
-367	val_367	2008-04-08	11
-325	val_325	2008-04-08	11
-167	val_167	2008-04-08	11
-195	val_195	2008-04-08	11
-475	val_475	2008-04-08	11
-17	val_17	2008-04-08	11
-113	val_113	2008-04-08	11
-155	val_155	2008-04-08	11
-203	val_203	2008-04-08	11
-339	val_339	2008-04-08	11
-0	val_0	2008-04-08	11
-455	val_455	2008-04-08	11
-128	val_128	2008-04-08	11
-311	val_311	2008-04-08	11
-316	val_316	2008-04-08	11
-57	val_57	2008-04-08	11
-302	val_302	2008-04-08	11
-205	val_205	2008-04-08	11
-149	val_149	2008-04-08	11
-438	val_438	2008-04-08	11
-345	val_345	2008-04-08	11
-129	val_129	2008-04-08	11
-170	val_170	2008-04-08	11
-20	val_20	2008-04-08	11
-489	val_489	2008-04-08	11
-157	val_157	2008-04-08	11
-378	val_378	2008-04-08	11
-221	val_221	2008-04-08	11
-92	val_92	2008-04-08	11
-111	val_111	2008-04-08	11
-47	val_47	2008-04-08	11
-72	val_72	2008-04-08	11
-4	val_4	2008-04-08	11
-280	val_280	2008-04-08	11
-35	val_35	2008-04-08	11
-427	val_427	2008-04-08	11
-277	val_277	2008-04-08	11
-208	val_208	2008-04-08	11
-356	val_356	2008-04-08	11
-399	val_399	2008-04-08	11
-169	val_169	2008-04-08	11
-382	val_382	2008-04-08	11
-498	val_498	2008-04-08	11
-125	val_125	2008-04-08	11
-386	val_386	2008-04-08	11
-437	val_437	2008-04-08	11
-469	val_469	2008-04-08	11
-192	val_192	2008-04-08	11
-286	val_286	2008-04-08	11
-187	val_187	2008-04-08	11
-176	val_176	2008-04-08	11
-54	val_54	2008-04-08	11
-459	val_459	2008-04-08	11
-51	val_51	2008-04-08	11
-138	val_138	2008-04-08	11
-103	val_103	2008-04-08	11
-239	val_239	2008-04-08	11
-213	val_213	2008-04-08	11
-216	val_216	2008-04-08	11
-430	val_430	2008-04-08	11
-278	val_278	2008-04-08	11
-176	val_176	2008-04-08	11
-289	val_289	2008-04-08	11
-221	val_221	2008-04-08	11
-65	val_65	2008-04-08	11
-318	val_318	2008-04-08	11
-332	val_332	2008-04-08	11
-311	val_311	2008-04-08	11
-275	val_275	2008-04-08	11
-137	val_137	2008-04-08	11
-241	val_241	2008-04-08	11
-83	val_83	2008-04-08	11
-333	val_333	2008-04-08	11
-180	val_180	2008-04-08	11
-284	val_284	2008-04-08	11
-12	val_12	2008-04-08	11
-230	val_230	2008-04-08	11
-181	val_181	2008-04-08	11
-67	val_67	2008-04-08	11
-260	val_260	2008-04-08	11
-404	val_404	2008-04-08	11
-384	val_384	2008-04-08	11
-489	val_489	2008-04-08	11
-353	val_353	2008-04-08	11
-373	val_373	2008-04-08	11
-272	val_272	2008-04-08	11
-138	val_138	2008-04-08	11
-217	val_217	2008-04-08	11
-84	val_84	2008-04-08	11
-348	val_348	2008-04-08	11
-466	val_466	2008-04-08	11
-58	val_58	2008-04-08	11
-8	val_8	2008-04-08	11
-411	val_411	2008-04-08	11
-230	val_230	2008-04-08	11
-208	val_208	2008-04-08	11
-348	val_348	2008-04-08	11
-24	val_24	2008-04-08	11
-463	val_463	2008-04-08	11
-431	val_431	2008-04-08	11
-179	val_179	2008-04-08	11
-172	val_172	2008-04-08	11
-42	val_42	2008-04-08	11
-129	val_129	2008-04-08	11
-158	val_158	2008-04-08	11
-119	val_119	2008-04-08	11
-496	val_496	2008-04-08	11
-0	val_0	2008-04-08	11
-322	val_322	2008-04-08	11
-197	val_197	2008-04-08	11
-468	val_468	2008-04-08	11
-393	val_393	2008-04-08	11
-454	val_454	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-199	val_199	2008-04-08	11
-191	val_191	2008-04-08	11
-418	val_418	2008-04-08	11
-96	val_96	2008-04-08	11
-26	val_26	2008-04-08	11
-165	val_165	2008-04-08	11
-327	val_327	2008-04-08	11
-230	val_230	2008-04-08	11
-205	val_205	2008-04-08	11
-120	val_120	2008-04-08	11
-131	val_131	2008-04-08	11
-51	val_51	2008-04-08	11
-404	val_404	2008-04-08	11
-43	val_43	2008-04-08	11
-436	val_436	2008-04-08	11
-156	val_156	2008-04-08	11
-469	val_469	2008-04-08	11
-468	val_468	2008-04-08	11
-308	val_308	2008-04-08	11
-95	val_95	2008-04-08	11
-196	val_196	2008-04-08	11
-288	val_288	2008-04-08	11
-481	val_481	2008-04-08	11
-457	val_457	2008-04-08	11
-98	val_98	2008-04-08	11
-282	val_282	2008-04-08	11
-197	val_197	2008-04-08	11
-187	val_187	2008-04-08	11
-318	val_318	2008-04-08	11
-318	val_318	2008-04-08	11
-409	val_409	2008-04-08	11
-470	val_470	2008-04-08	11
-137	val_137	2008-04-08	11
-369	val_369	2008-04-08	11
-316	val_316	2008-04-08	11
-169	val_169	2008-04-08	11
-413	val_413	2008-04-08	11
-85	val_85	2008-04-08	11
-77	val_77	2008-04-08	11
-0	val_0	2008-04-08	11
-490	val_490	2008-04-08	11
-87	val_87	2008-04-08	11
-364	val_364	2008-04-08	11
-179	val_179	2008-04-08	11
-118	val_118	2008-04-08	11
-134	val_134	2008-04-08	11
-395	val_395	2008-04-08	11
-282	val_282	2008-04-08	11
-138	val_138	2008-04-08	11
-238	val_238	2008-04-08	11
-419	val_419	2008-04-08	11
-15	val_15	2008-04-08	11
-118	val_118	2008-04-08	11
-72	val_72	2008-04-08	11
-90	val_90	2008-04-08	11
-307	val_307	2008-04-08	11
-19	val_19	2008-04-08	11
-435	val_435	2008-04-08	11
-10	val_10	2008-04-08	11
-277	val_277	2008-04-08	11
-273	val_273	2008-04-08	11
-306	val_306	2008-04-08	11
-224	val_224	2008-04-08	11
-309	val_309	2008-04-08	11
-389	val_389	2008-04-08	11
-327	val_327	2008-04-08	11
-242	val_242	2008-04-08	11
-369	val_369	2008-04-08	11
-392	val_392	2008-04-08	11
-272	val_272	2008-04-08	11
-331	val_331	2008-04-08	11
-401	val_401	2008-04-08	11
-242	val_242	2008-04-08	11
-452	val_452	2008-04-08	11
-177	val_177	2008-04-08	11
-226	val_226	2008-04-08	11
-5	val_5	2008-04-08	11
-497	val_497	2008-04-08	11
-402	val_402	2008-04-08	11
-396	val_396	2008-04-08	11
-317	val_317	2008-04-08	11
-395	val_395	2008-04-08	11
-58	val_58	2008-04-08	11
-35	val_35	2008-04-08	11
-336	val_336	2008-04-08	11
-95	val_95	2008-04-08	11
-11	val_11	2008-04-08	11
-168	val_168	2008-04-08	11
-34	val_34	2008-04-08	11
-229	val_229	2008-04-08	11
-233	val_233	2008-04-08	11
-143	val_143	2008-04-08	11
-472	val_472	2008-04-08	11
-322	val_322	2008-04-08	11
-498	val_498	2008-04-08	11
-160	val_160	2008-04-08	11
-195	val_195	2008-04-08	11
-42	val_42	2008-04-08	11
-321	val_321	2008-04-08	11
-430	val_430	2008-04-08	11
-119	val_119	2008-04-08	11
-489	val_489	2008-04-08	11
-458	val_458	2008-04-08	11
-78	val_78	2008-04-08	11
-76	val_76	2008-04-08	11
-41	val_41	2008-04-08	11
-223	val_223	2008-04-08	11
-492	val_492	2008-04-08	11
-149	val_149	2008-04-08	11
-449	val_449	2008-04-08	11
-218	val_218	2008-04-08	11
-228	val_228	2008-04-08	11
-138	val_138	2008-04-08	11
-453	val_453	2008-04-08	11
-30	val_30	2008-04-08	11
-209	val_209	2008-04-08	11
-64	val_64	2008-04-08	11
-468	val_468	2008-04-08	11
-76	val_76	2008-04-08	11
-74	val_74	2008-04-08	11
-342	val_342	2008-04-08	11
-69	val_69	2008-04-08	11
-230	val_230	2008-04-08	11
-33	val_33	2008-04-08	11
-368	val_368	2008-04-08	11
-103	val_103	2008-04-08	11
-296	val_296	2008-04-08	11
-113	val_113	2008-04-08	11
-216	val_216	2008-04-08	11
-367	val_367	2008-04-08	11
-344	val_344	2008-04-08	11
-167	val_167	2008-04-08	11
-274	val_274	2008-04-08	11
-219	val_219	2008-04-08	11
-239	val_239	2008-04-08	11
-485	val_485	2008-04-08	11
-116	val_116	2008-04-08	11
-223	val_223	2008-04-08	11
-256	val_256	2008-04-08	11
-263	val_263	2008-04-08	11
-70	val_70	2008-04-08	11
-487	val_487	2008-04-08	11
-480	val_480	2008-04-08	11
-401	val_401	2008-04-08	11
-288	val_288	2008-04-08	11
-191	val_191	2008-04-08	11
-5	val_5	2008-04-08	11
-244	val_244	2008-04-08	11
-438	val_438	2008-04-08	11
-128	val_128	2008-04-08	11
-467	val_467	2008-04-08	11
-432	val_432	2008-04-08	11
-202	val_202	2008-04-08	11
-316	val_316	2008-04-08	11
-229	val_229	2008-04-08	11
-469	val_469	2008-04-08	11
-463	val_463	2008-04-08	11
-280	val_280	2008-04-08	11
-2	val_2	2008-04-08	11
-35	val_35	2008-04-08	11
-283	val_283	2008-04-08	11
-331	val_331	2008-04-08	11
-235	val_235	2008-04-08	11
-80	val_80	2008-04-08	11
-44	val_44	2008-04-08	11
-193	val_193	2008-04-08	11
-321	val_321	2008-04-08	11
-335	val_335	2008-04-08	11
-104	val_104	2008-04-08	11
-466	val_466	2008-04-08	11
-366	val_366	2008-04-08	11
-175	val_175	2008-04-08	11
-403	val_403	2008-04-08	11
-483	val_483	2008-04-08	11
-53	val_53	2008-04-08	11
-105	val_105	2008-04-08	11
-257	val_257	2008-04-08	11
-406	val_406	2008-04-08	11
-409	val_409	2008-04-08	11
-190	val_190	2008-04-08	11
-406	val_406	2008-04-08	11
-401	val_401	2008-04-08	11
-114	val_114	2008-04-08	11
-258	val_258	2008-04-08	11
-90	val_90	2008-04-08	11
-203	val_203	2008-04-08	11
-262	val_262	2008-04-08	11
-348	val_348	2008-04-08	11
-424	val_424	2008-04-08	11
-12	val_12	2008-04-08	11
-396	val_396	2008-04-08	11
-201	val_201	2008-04-08	11
-217	val_217	2008-04-08	11
-164	val_164	2008-04-08	11
-431	val_431	2008-04-08	11
-454	val_454	2008-04-08	11
-478	val_478	2008-04-08	11
-298	val_298	2008-04-08	11
-125	val_125	2008-04-08	11
-431	val_431	2008-04-08	11
-164	val_164	2008-04-08	11
-424	val_424	2008-04-08	11
-187	val_187	2008-04-08	11
-382	val_382	2008-04-08	11
-5	val_5	2008-04-08	11
-70	val_70	2008-04-08	11
-397	val_397	2008-04-08	11
-480	val_480	2008-04-08	11
-291	val_291	2008-04-08	11
-24	val_24	2008-04-08	11
-351	val_351	2008-04-08	11
-255	val_255	2008-04-08	11
-104	val_104	2008-04-08	11
-70	val_70	2008-04-08	11
-163	val_163	2008-04-08	11
-438	val_438	2008-04-08	11
-119	val_119	2008-04-08	11
-414	val_414	2008-04-08	11
-200	val_200	2008-04-08	11
-491	val_491	2008-04-08	11
-237	val_237	2008-04-08	11
-439	val_439	2008-04-08	11
-360	val_360	2008-04-08	11
-248	val_248	2008-04-08	11
-479	val_479	2008-04-08	11
-305	val_305	2008-04-08	11
-417	val_417	2008-04-08	11
-199	val_199	2008-04-08	11
-444	val_444	2008-04-08	11
-120	val_120	2008-04-08	11
-429	val_429	2008-04-08	11
-169	val_169	2008-04-08	11
-443	val_443	2008-04-08	11
-323	val_323	2008-04-08	11
-325	val_325	2008-04-08	11
-277	val_277	2008-04-08	11
-230	val_230	2008-04-08	11
-478	val_478	2008-04-08	11
-178	val_178	2008-04-08	11
-468	val_468	2008-04-08	11
-310	val_310	2008-04-08	11
-317	val_317	2008-04-08	11
-333	val_333	2008-04-08	11
-493	val_493	2008-04-08	11
-460	val_460	2008-04-08	11
-207	val_207	2008-04-08	11
-249	val_249	2008-04-08	11
-265	val_265	2008-04-08	11
-480	val_480	2008-04-08	11
-83	val_83	2008-04-08	11
-136	val_136	2008-04-08	11
-353	val_353	2008-04-08	11
-172	val_172	2008-04-08	11
-214	val_214	2008-04-08	11
-462	val_462	2008-04-08	11
-233	val_233	2008-04-08	11
-406	val_406	2008-04-08	11
-133	val_133	2008-04-08	11
-175	val_175	2008-04-08	11
-189	val_189	2008-04-08	11
-454	val_454	2008-04-08	11
-375	val_375	2008-04-08	11
-401	val_401	2008-04-08	11
-421	val_421	2008-04-08	11
-407	val_407	2008-04-08	11
-384	val_384	2008-04-08	11
-256	val_256	2008-04-08	11
-26	val_26	2008-04-08	11
-134	val_134	2008-04-08	11
-67	val_67	2008-04-08	11
-384	val_384	2008-04-08	11
-379	val_379	2008-04-08	11
-18	val_18	2008-04-08	11
-462	val_462	2008-04-08	11
-492	val_492	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-9	val_9	2008-04-08	11
-341	val_341	2008-04-08	11
-498	val_498	2008-04-08	11
-146	val_146	2008-04-08	11
-458	val_458	2008-04-08	11
-362	val_362	2008-04-08	11
-186	val_186	2008-04-08	11
-285	val_285	2008-04-08	11
-348	val_348	2008-04-08	11
-167	val_167	2008-04-08	11
-18	val_18	2008-04-08	11
-273	val_273	2008-04-08	11
-183	val_183	2008-04-08	11
-281	val_281	2008-04-08	11
-344	val_344	2008-04-08	11
-97	val_97	2008-04-08	11
-469	val_469	2008-04-08	11
-315	val_315	2008-04-08	11
-84	val_84	2008-04-08	11
-28	val_28	2008-04-08	11
-37	val_37	2008-04-08	11
-448	val_448	2008-04-08	11
-152	val_152	2008-04-08	11
-348	val_348	2008-04-08	11
-307	val_307	2008-04-08	11
-194	val_194	2008-04-08	11
-414	val_414	2008-04-08	11
-477	val_477	2008-04-08	11
-222	val_222	2008-04-08	11
-126	val_126	2008-04-08	11
-90	val_90	2008-04-08	11
-169	val_169	2008-04-08	11
-403	val_403	2008-04-08	11
-400	val_400	2008-04-08	11
-200	val_200	2008-04-08	11
-97	val_97	2008-04-08	11
-238	val_238	2008-04-08	12
-86	val_86	2008-04-08	12
-311	val_311	2008-04-08	12
-27	val_27	2008-04-08	12
-165	val_165	2008-04-08	12
-409	val_409	2008-04-08	12
-255	val_255	2008-04-08	12
-278	val_278	2008-04-08	12
-98	val_98	2008-04-08	12
-484	val_484	2008-04-08	12
-265	val_265	2008-04-08	12
-193	val_193	2008-04-08	12
-401	val_401	2008-04-08	12
-150	val_150	2008-04-08	12
-273	val_273	2008-04-08	12
-224	val_224	2008-04-08	12
-369	val_369	2008-04-08	12
-66	val_66	2008-04-08	12
-128	val_128	2008-04-08	12
-213	val_213	2008-04-08	12
-146	val_146	2008-04-08	12
-406	val_406	2008-04-08	12
-429	val_429	2008-04-08	12
-374	val_374	2008-04-08	12
-152	val_152	2008-04-08	12
-469	val_469	2008-04-08	12
-145	val_145	2008-04-08	12
-495	val_495	2008-04-08	12
-37	val_37	2008-04-08	12
-327	val_327	2008-04-08	12
-281	val_281	2008-04-08	12
-277	val_277	2008-04-08	12
-209	val_209	2008-04-08	12
-15	val_15	2008-04-08	12
-82	val_82	2008-04-08	12
-403	val_403	2008-04-08	12
-166	val_166	2008-04-08	12
-417	val_417	2008-04-08	12
-430	val_430	2008-04-08	12
-252	val_252	2008-04-08	12
-292	val_292	2008-04-08	12
-219	val_219	2008-04-08	12
-287	val_287	2008-04-08	12
-153	val_153	2008-04-08	12
-193	val_193	2008-04-08	12
-338	val_338	2008-04-08	12
-446	val_446	2008-04-08	12
-459	val_459	2008-04-08	12
-394	val_394	2008-04-08	12
-237	val_237	2008-04-08	12
-482	val_482	2008-04-08	12
-174	val_174	2008-04-08	12
-413	val_413	2008-04-08	12
-494	val_494	2008-04-08	12
-207	val_207	2008-04-08	12
-199	val_199	2008-04-08	12
-466	val_466	2008-04-08	12
-208	val_208	2008-04-08	12
-174	val_174	2008-04-08	12
-399	val_399	2008-04-08	12
-396	val_396	2008-04-08	12
-247	val_247	2008-04-08	12
-417	val_417	2008-04-08	12
-489	val_489	2008-04-08	12
-162	val_162	2008-04-08	12
-377	val_377	2008-04-08	12
-397	val_397	2008-04-08	12
-309	val_309	2008-04-08	12
-365	val_365	2008-04-08	12
-266	val_266	2008-04-08	12
-439	val_439	2008-04-08	12
-342	val_342	2008-04-08	12
-367	val_367	2008-04-08	12
-325	val_325	2008-04-08	12
-167	val_167	2008-04-08	12
-195	val_195	2008-04-08	12
-475	val_475	2008-04-08	12
-17	val_17	2008-04-08	12
-113	val_113	2008-04-08	12
-155	val_155	2008-04-08	12
-203	val_203	2008-04-08	12
-339	val_339	2008-04-08	12
-0	val_0	2008-04-08	12
-455	val_455	2008-04-08	12
-128	val_128	2008-04-08	12
-311	val_311	2008-04-08	12
-316	val_316	2008-04-08	12
-57	val_57	2008-04-08	12
-302	val_302	2008-04-08	12
-205	val_205	2008-04-08	12
-149	val_149	2008-04-08	12
-438	val_438	2008-04-08	12
-345	val_345	2008-04-08	12
-129	val_129	2008-04-08	12
-170	val_170	2008-04-08	12
-20	val_20	2008-04-08	12
-489	val_489	2008-04-08	12
-157	val_157	2008-04-08	12
-378	val_378	2008-04-08	12
-221	val_221	2008-04-08	12
-92	val_92	2008-04-08	12
-111	val_111	2008-04-08	12
-47	val_47	2008-04-08	12
-72	val_72	2008-04-08	12
-4	val_4	2008-04-08	12
-280	val_280	2008-04-08	12
-35	val_35	2008-04-08	12
-427	val_427	2008-04-08	12
-277	val_277	2008-04-08	12
-208	val_208	2008-04-08	12
-356	val_356	2008-04-08	12
-399	val_399	2008-04-08	12
-169	val_169	2008-04-08	12
-382	val_382	2008-04-08	12
-498	val_498	2008-04-08	12
-125	val_125	2008-04-08	12
-386	val_386	2008-04-08	12
-437	val_437	2008-04-08	12
-469	val_469	2008-04-08	12
-192	val_192	2008-04-08	12
-286	val_286	2008-04-08	12
-187	val_187	2008-04-08	12
-176	val_176	2008-04-08	12
-54	val_54	2008-04-08	12
-459	val_459	2008-04-08	12
-51	val_51	2008-04-08	12
-138	val_138	2008-04-08	12
-103	val_103	2008-04-08	12
-239	val_239	2008-04-08	12
-213	val_213	2008-04-08	12
-216	val_216	2008-04-08	12
-430	val_430	2008-04-08	12
-278	val_278	2008-04-08	12
-176	val_176	2008-04-08	12
-289	val_289	2008-04-08	12
-221	val_221	2008-04-08	12
-65	val_65	2008-04-08	12
-318	val_318	2008-04-08	12
-332	val_332	2008-04-08	12
-311	val_311	2008-04-08	12
-275	val_275	2008-04-08	12
-137	val_137	2008-04-08	12
-241	val_241	2008-04-08	12
-83	val_83	2008-04-08	12
-333	val_333	2008-04-08	12
-180	val_180	2008-04-08	12
-284	val_284	2008-04-08	12
-12	val_12	2008-04-08	12
-230	val_230	2008-04-08	12
-181	val_181	2008-04-08	12
-67	val_67	2008-04-08	12
-260	val_260	2008-04-08	12
-404	val_404	2008-04-08	12
-384	val_384	2008-04-08	12
-489	val_489	2008-04-08	12
-353	val_353	2008-04-08	12
-373	val_373	2008-04-08	12
-272	val_272	2008-04-08	12
-138	val_138	2008-04-08	12
-217	val_217	2008-04-08	12
-84	val_84	2008-04-08	12
-348	val_348	2008-04-08	12
-466	val_466	2008-04-08	12
-58	val_58	2008-04-08	12
-8	val_8	2008-04-08	12
-411	val_411	2008-04-08	12
-230	val_230	2008-04-08	12
-208	val_208	2008-04-08	12
-348	val_348	2008-04-08	12
-24	val_24	2008-04-08	12
-463	val_463	2008-04-08	12
-431	val_431	2008-04-08	12
-179	val_179	2008-04-08	12
-172	val_172	2008-04-08	12
-42	val_42	2008-04-08	12
-129	val_129	2008-04-08	12
-158	val_158	2008-04-08	12
-119	val_119	2008-04-08	12
-496	val_496	2008-04-08	12
-0	val_0	2008-04-08	12
-322	val_322	2008-04-08	12
-197	val_197	2008-04-08	12
-468	val_468	2008-04-08	12
-393	val_393	2008-04-08	12
-454	val_454	2008-04-08	12
-100	val_100	2008-04-08	12
-298	val_298	2008-04-08	12
-199	val_199	2008-04-08	12
-191	val_191	2008-04-08	12
-418	val_418	2008-04-08	12
-96	val_96	2008-04-08	12
-26	val_26	2008-04-08	12
-165	val_165	2008-04-08	12
-327	val_327	2008-04-08	12
-230	val_230	2008-04-08	12
-205	val_205	2008-04-08	12
-120	val_120	2008-04-08	12
-131	val_131	2008-04-08	12
-51	val_51	2008-04-08	12
-404	val_404	2008-04-08	12
-43	val_43	2008-04-08	12
-436	val_436	2008-04-08	12
-156	val_156	2008-04-08	12
-469	val_469	2008-04-08	12
-468	val_468	2008-04-08	12
-308	val_308	2008-04-08	12
-95	val_95	2008-04-08	12
-196	val_196	2008-04-08	12
-288	val_288	2008-04-08	12
-481	val_481	2008-04-08	12
-457	val_457	2008-04-08	12
-98	val_98	2008-04-08	12
-282	val_282	2008-04-08	12
-197	val_197	2008-04-08	12
-187	val_187	2008-04-08	12
-318	val_318	2008-04-08	12
-318	val_318	2008-04-08	12
-409	val_409	2008-04-08	12
-470	val_470	2008-04-08	12
-137	val_137	2008-04-08	12
-369	val_369	2008-04-08	12
-316	val_316	2008-04-08	12
-169	val_169	2008-04-08	12
-413	val_413	2008-04-08	12
-85	val_85	2008-04-08	12
-77	val_77	2008-04-08	12
-0	val_0	2008-04-08	12
-490	val_490	2008-04-08	12
-87	val_87	2008-04-08	12
-364	val_364	2008-04-08	12
-179	val_179	2008-04-08	12
-118	val_118	2008-04-08	12
-134	val_134	2008-04-08	12
-395	val_395	2008-04-08	12
-282	val_282	2008-04-08	12
-138	val_138	2008-04-08	12
-238	val_238	2008-04-08	12
-419	val_419	2008-04-08	12
-15	val_15	2008-04-08	12
-118	val_118	2008-04-08	12
-72	val_72	2008-04-08	12
-90	val_90	2008-04-08	12
-307	val_307	2008-04-08	12
-19	val_19	2008-04-08	12
-435	val_435	2008-04-08	12
-10	val_10	2008-04-08	12
-277	val_277	2008-04-08	12
-273	val_273	2008-04-08	12
-306	val_306	2008-04-08	12
-224	val_224	2008-04-08	12
-309	val_309	2008-04-08	12
-389	val_389	2008-04-08	12
-327	val_327	2008-04-08	12
-242	val_242	2008-04-08	12
-369	val_369	2008-04-08	12
-392	val_392	2008-04-08	12
-272	val_272	2008-04-08	12
-331	val_331	2008-04-08	12
-401	val_401	2008-04-08	12
-242	val_242	2008-04-08	12
-452	val_452	2008-04-08	12
-177	val_177	2008-04-08	12
-226	val_226	2008-04-08	12
-5	val_5	2008-04-08	12
-497	val_497	2008-04-08	12
-402	val_402	2008-04-08	12
-396	val_396	2008-04-08	12
-317	val_317	2008-04-08	12
-395	val_395	2008-04-08	12
-58	val_58	2008-04-08	12
-35	val_35	2008-04-08	12
-336	val_336	2008-04-08	12
-95	val_95	2008-04-08	12
-11	val_11	2008-04-08	12
-168	val_168	2008-04-08	12
-34	val_34	2008-04-08	12
-229	val_229	2008-04-08	12
-233	val_233	2008-04-08	12
-143	val_143	2008-04-08	12
-472	val_472	2008-04-08	12
-322	val_322	2008-04-08	12
-498	val_498	2008-04-08	12
-160	val_160	2008-04-08	12
-195	val_195	2008-04-08	12
-42	val_42	2008-04-08	12
-321	val_321	2008-04-08	12
-430	val_430	2008-04-08	12
-119	val_119	2008-04-08	12
-489	val_489	2008-04-08	12
-458	val_458	2008-04-08	12
-78	val_78	2008-04-08	12
-76	val_76	2008-04-08	12
-41	val_41	2008-04-08	12
-223	val_223	2008-04-08	12
-492	val_492	2008-04-08	12
-149	val_149	2008-04-08	12
-449	val_449	2008-04-08	12
-218	val_218	2008-04-08	12
-228	val_228	2008-04-08	12
-138	val_138	2008-04-08	12
-453	val_453	2008-04-08	12
-30	val_30	2008-04-08	12
-209	val_209	2008-04-08	12
-64	val_64	2008-04-08	12
-468	val_468	2008-04-08	12
-76	val_76	2008-04-08	12
-74	val_74	2008-04-08	12
-342	val_342	2008-04-08	12
-69	val_69	2008-04-08	12
-230	val_230	2008-04-08	12
-33	val_33	2008-04-08	12
-368	val_368	2008-04-08	12
-103	val_103	2008-04-08	12
-296	val_296	2008-04-08	12
-113	val_113	2008-04-08	12
-216	val_216	2008-04-08	12
-367	val_367	2008-04-08	12
-344	val_344	2008-04-08	12
-167	val_167	2008-04-08	12
-274	val_274	2008-04-08	12
-219	val_219	2008-04-08	12
-239	val_239	2008-04-08	12
-485	val_485	2008-04-08	12
-116	val_116	2008-04-08	12
-223	val_223	2008-04-08	12
-256	val_256	2008-04-08	12
-263	val_263	2008-04-08	12
-70	val_70	2008-04-08	12
-487	val_487	2008-04-08	12
-480	val_480	2008-04-08	12
-401	val_401	2008-04-08	12
-288	val_288	2008-04-08	12
-191	val_191	2008-04-08	12
-5	val_5	2008-04-08	12
-244	val_244	2008-04-08	12
-438	val_438	2008-04-08	12
-128	val_128	2008-04-08	12
-467	val_467	2008-04-08	12
-432	val_432	2008-04-08	12
-202	val_202	2008-04-08	12
-316	val_316	2008-04-08	12
-229	val_229	2008-04-08	12
-469	val_469	2008-04-08	12
-463	val_463	2008-04-08	12
-280	val_280	2008-04-08	12
-2	val_2	2008-04-08	12
-35	val_35	2008-04-08	12
-283	val_283	2008-04-08	12
-331	val_331	2008-04-08	12
-235	val_235	2008-04-08	12
-80	val_80	2008-04-08	12
-44	val_44	2008-04-08	12
-193	val_193	2008-04-08	12
-321	val_321	2008-04-08	12
-335	val_335	2008-04-08	12
-104	val_104	2008-04-08	12
-466	val_466	2008-04-08	12
-366	val_366	2008-04-08	12
-175	val_175	2008-04-08	12
-403	val_403	2008-04-08	12
-483	val_483	2008-04-08	12
-53	val_53	2008-04-08	12
-105	val_105	2008-04-08	12
-257	val_257	2008-04-08	12
-406	val_406	2008-04-08	12
-409	val_409	2008-04-08	12
-190	val_190	2008-04-08	12
-406	val_406	2008-04-08	12
-401	val_401	2008-04-08	12
-114	val_114	2008-04-08	12
-258	val_258	2008-04-08	12
-90	val_90	2008-04-08	12
-203	val_203	2008-04-08	12
-262	val_262	2008-04-08	12
-348	val_348	2008-04-08	12
-424	val_424	2008-04-08	12
-12	val_12	2008-04-08	12
-396	val_396	2008-04-08	12
-201	val_201	2008-04-08	12
-217	val_217	2008-04-08	12
-164	val_164	2008-04-08	12
-431	val_431	2008-04-08	12
-454	val_454	2008-04-08	12
-478	val_478	2008-04-08	12
-298	val_298	2008-04-08	12
-125	val_125	2008-04-08	12
-431	val_431	2008-04-08	12
-164	val_164	2008-04-08	12
-424	val_424	2008-04-08	12
-187	val_187	2008-04-08	12
-382	val_382	2008-04-08	12
-5	val_5	2008-04-08	12
-70	val_70	2008-04-08	12
-397	val_397	2008-04-08	12
-480	val_480	2008-04-08	12
-291	val_291	2008-04-08	12
-24	val_24	2008-04-08	12
-351	val_351	2008-04-08	12
-255	val_255	2008-04-08	12
-104	val_104	2008-04-08	12
-70	val_70	2008-04-08	12
-163	val_163	2008-04-08	12
-438	val_438	2008-04-08	12
-119	val_119	2008-04-08	12
-414	val_414	2008-04-08	12
-200	val_200	2008-04-08	12
-491	val_491	2008-04-08	12
-237	val_237	2008-04-08	12
-439	val_439	2008-04-08	12
-360	val_360	2008-04-08	12
-248	val_248	2008-04-08	12
-479	val_479	2008-04-08	12
-305	val_305	2008-04-08	12
-417	val_417	2008-04-08	12
-199	val_199	2008-04-08	12
-444	val_444	2008-04-08	12
-120	val_120	2008-04-08	12
-429	val_429	2008-04-08	12
-169	val_169	2008-04-08	12
-443	val_443	2008-04-08	12
-323	val_323	2008-04-08	12
-325	val_325	2008-04-08	12
-277	val_277	2008-04-08	12
-230	val_230	2008-04-08	12
-478	val_478	2008-04-08	12
-178	val_178	2008-04-08	12
-468	val_468	2008-04-08	12
-310	val_310	2008-04-08	12
-317	val_317	2008-04-08	12
-333	val_333	2008-04-08	12
-493	val_493	2008-04-08	12
-460	val_460	2008-04-08	12
-207	val_207	2008-04-08	12
-249	val_249	2008-04-08	12
-265	val_265	2008-04-08	12
-480	val_480	2008-04-08	12
-83	val_83	2008-04-08	12
-136	val_136	2008-04-08	12
-353	val_353	2008-04-08	12
-172	val_172	2008-04-08	12
-214	val_214	2008-04-08	12
-462	val_462	2008-04-08	12
-233	val_233	2008-04-08	12
-406	val_406	2008-04-08	12
-133	val_133	2008-04-08	12
-175	val_175	2008-04-08	12
-189	val_189	2008-04-08	12
-454	val_454	2008-04-08	12
-375	val_375	2008-04-08	12
-401	val_401	2008-04-08	12
-421	val_421	2008-04-08	12
-407	val_407	2008-04-08	12
-384	val_384	2008-04-08	12
-256	val_256	2008-04-08	12
-26	val_26	2008-04-08	12
-134	val_134	2008-04-08	12
-67	val_67	2008-04-08	12
-384	val_384	2008-04-08	12
-379	val_379	2008-04-08	12
-18	val_18	2008-04-08	12
-462	val_462	2008-04-08	12
-492	val_492	2008-04-08	12
-100	val_100	2008-04-08	12
-298	val_298	2008-04-08	12
-9	val_9	2008-04-08	12
-341	val_341	2008-04-08	12
-498	val_498	2008-04-08	12
-146	val_146	2008-04-08	12
-458	val_458	2008-04-08	12
-362	val_362	2008-04-08	12
-186	val_186	2008-04-08	12
-285	val_285	2008-04-08	12
-348	val_348	2008-04-08	12
-167	val_167	2008-04-08	12
-18	val_18	2008-04-08	12
-273	val_273	2008-04-08	12
-183	val_183	2008-04-08	12
-281	val_281	2008-04-08	12
-344	val_344	2008-04-08	12
-97	val_97	2008-04-08	12
-469	val_469	2008-04-08	12
-315	val_315	2008-04-08	12
-84	val_84	2008-04-08	12
-28	val_28	2008-04-08	12
-37	val_37	2008-04-08	12
-448	val_448	2008-04-08	12
-152	val_152	2008-04-08	12
-348	val_348	2008-04-08	12
-307	val_307	2008-04-08	12
-194	val_194	2008-04-08	12
-414	val_414	2008-04-08	12
-477	val_477	2008-04-08	12
-222	val_222	2008-04-08	12
-126	val_126	2008-04-08	12
-90	val_90	2008-04-08	12
-169	val_169	2008-04-08	12
-403	val_403	2008-04-08	12
-400	val_400	2008-04-08	12
-200	val_200	2008-04-08	12
-97	val_97	2008-04-08	12
-238	val_238	2008-12-31	11
-86	val_86	2008-12-31	11
-311	val_311	2008-12-31	11
-27	val_27	2008-12-31	11
-165	val_165	2008-12-31	11
-409	val_409	2008-12-31	11
-255	val_255	2008-12-31	11
-278	val_278	2008-12-31	11
-98	val_98	2008-12-31	11
-484	val_484	2008-12-31	11
-265	val_265	2008-12-31	11
-193	val_193	2008-12-31	11
-401	val_401	2008-12-31	11
-150	val_150	2008-12-31	11
-273	val_273	2008-12-31	11
-224	val_224	2008-12-31	11
-369	val_369	2008-12-31	11
-66	val_66	2008-12-31	11
-128	val_128	2008-12-31	11
-213	val_213	2008-12-31	11
-146	val_146	2008-12-31	11
-406	val_406	2008-12-31	11
-429	val_429	2008-12-31	11
-374	val_374	2008-12-31	11
-152	val_152	2008-12-31	11
-469	val_469	2008-12-31	11
-145	val_145	2008-12-31	11
-495	val_495	2008-12-31	11
-37	val_37	2008-12-31	11
-327	val_327	2008-12-31	11
-281	val_281	2008-12-31	11
-277	val_277	2008-12-31	11
-209	val_209	2008-12-31	11
-15	val_15	2008-12-31	11
-82	val_82	2008-12-31	11
-403	val_403	2008-12-31	11
-166	val_166	2008-12-31	11
-417	val_417	2008-12-31	11
-430	val_430	2008-12-31	11
-252	val_252	2008-12-31	11
-292	val_292	2008-12-31	11
-219	val_219	2008-12-31	11
-287	val_287	2008-12-31	11
-153	val_153	2008-12-31	11
-193	val_193	2008-12-31	11
-338	val_338	2008-12-31	11
-446	val_446	2008-12-31	11
-459	val_459	2008-12-31	11
-394	val_394	2008-12-31	11
-237	val_237	2008-12-31	11
-482	val_482	2008-12-31	11
-174	val_174	2008-12-31	11
-413	val_413	2008-12-31	11
-494	val_494	2008-12-31	11
-207	val_207	2008-12-31	11
-199	val_199	2008-12-31	11
-466	val_466	2008-12-31	11
-208	val_208	2008-12-31	11
-174	val_174	2008-12-31	11
-399	val_399	2008-12-31	11
-396	val_396	2008-12-31	11
-247	val_247	2008-12-31	11
-417	val_417	2008-12-31	11
-489	val_489	2008-12-31	11
-162	val_162	2008-12-31	11
-377	val_377	2008-12-31	11
-397	val_397	2008-12-31	11
-309	val_309	2008-12-31	11
-365	val_365	2008-12-31	11
-266	val_266	2008-12-31	11
-439	val_439	2008-12-31	11
-342	val_342	2008-12-31	11
-367	val_367	2008-12-31	11
-325	val_325	2008-12-31	11
-167	val_167	2008-12-31	11
-195	val_195	2008-12-31	11
-475	val_475	2008-12-31	11
-17	val_17	2008-12-31	11
-113	val_113	2008-12-31	11
-155	val_155	2008-12-31	11
-203	val_203	2008-12-31	11
-339	val_339	2008-12-31	11
-0	val_0	2008-12-31	11
-455	val_455	2008-12-31	11
-128	val_128	2008-12-31	11
-311	val_311	2008-12-31	11
-316	val_316	2008-12-31	11
-57	val_57	2008-12-31	11
-302	val_302	2008-12-31	11
-205	val_205	2008-12-31	11
-149	val_149	2008-12-31	11
-438	val_438	2008-12-31	11
-345	val_345	2008-12-31	11
-129	val_129	2008-12-31	11
-170	val_170	2008-12-31	11
-20	val_20	2008-12-31	11
-489	val_489	2008-12-31	11
-157	val_157	2008-12-31	11
-378	val_378	2008-12-31	11
-221	val_221	2008-12-31	11
-92	val_92	2008-12-31	11
-111	val_111	2008-12-31	11
-47	val_47	2008-12-31	11
-72	val_72	2008-12-31	11
-4	val_4	2008-12-31	11
-280	val_280	2008-12-31	11
-35	val_35	2008-12-31	11
-427	val_427	2008-12-31	11
-277	val_277	2008-12-31	11
-208	val_208	2008-12-31	11
-356	val_356	2008-12-31	11
-399	val_399	2008-12-31	11
-169	val_169	2008-12-31	11
-382	val_382	2008-12-31	11
-498	val_498	2008-12-31	11
-125	val_125	2008-12-31	11
-386	val_386	2008-12-31	11
-437	val_437	2008-12-31	11
-469	val_469	2008-12-31	11
-192	val_192	2008-12-31	11
-286	val_286	2008-12-31	11
-187	val_187	2008-12-31	11
-176	val_176	2008-12-31	11
-54	val_54	2008-12-31	11
-459	val_459	2008-12-31	11
-51	val_51	2008-12-31	11
-138	val_138	2008-12-31	11
-103	val_103	2008-12-31	11
-239	val_239	2008-12-31	11
-213	val_213	2008-12-31	11
-216	val_216	2008-12-31	11
-430	val_430	2008-12-31	11
-278	val_278	2008-12-31	11
-176	val_176	2008-12-31	11
-289	val_289	2008-12-31	11
-221	val_221	2008-12-31	11
-65	val_65	2008-12-31	11
-318	val_318	2008-12-31	11
-332	val_332	2008-12-31	11
-311	val_311	2008-12-31	11
-275	val_275	2008-12-31	11
-137	val_137	2008-12-31	11
-241	val_241	2008-12-31	11
-83	val_83	2008-12-31	11
-333	val_333	2008-12-31	11
-180	val_180	2008-12-31	11
-284	val_284	2008-12-31	11
-12	val_12	2008-12-31	11
-230	val_230	2008-12-31	11
-181	val_181	2008-12-31	11
-67	val_67	2008-12-31	11
-260	val_260	2008-12-31	11
-404	val_404	2008-12-31	11
-384	val_384	2008-12-31	11
-489	val_489	2008-12-31	11
-353	val_353	2008-12-31	11
-373	val_373	2008-12-31	11
-272	val_272	2008-12-31	11
-138	val_138	2008-12-31	11
-217	val_217	2008-12-31	11
-84	val_84	2008-12-31	11
-348	val_348	2008-12-31	11
-466	val_466	2008-12-31	11
-58	val_58	2008-12-31	11
-8	val_8	2008-12-31	11
-411	val_411	2008-12-31	11
-230	val_230	2008-12-31	11
-208	val_208	2008-12-31	11
-348	val_348	2008-12-31	11
-24	val_24	2008-12-31	11
-463	val_463	2008-12-31	11
-431	val_431	2008-12-31	11
-179	val_179	2008-12-31	11
-172	val_172	2008-12-31	11
-42	val_42	2008-12-31	11
-129	val_129	2008-12-31	11
-158	val_158	2008-12-31	11
-119	val_119	2008-12-31	11
-496	val_496	2008-12-31	11
-0	val_0	2008-12-31	11
-322	val_322	2008-12-31	11
-197	val_197	2008-12-31	11
-468	val_468	2008-12-31	11
-393	val_393	2008-12-31	11
-454	val_454	2008-12-31	11
-100	val_100	2008-12-31	11
-298	val_298	2008-12-31	11
-199	val_199	2008-12-31	11
-191	val_191	2008-12-31	11
-418	val_418	2008-12-31	11
-96	val_96	2008-12-31	11
-26	val_26	2008-12-31	11
-165	val_165	2008-12-31	11
-327	val_327	2008-12-31	11
-230	val_230	2008-12-31	11
-205	val_205	2008-12-31	11
-120	val_120	2008-12-31	11
-131	val_131	2008-12-31	11
-51	val_51	2008-12-31	11
-404	val_404	2008-12-31	11
-43	val_43	2008-12-31	11
-436	val_436	2008-12-31	11
-156	val_156	2008-12-31	11
-469	val_469	2008-12-31	11
-468	val_468	2008-12-31	11
-308	val_308	2008-12-31	11
-95	val_95	2008-12-31	11
-196	val_196	2008-12-31	11
-288	val_288	2008-12-31	11
-481	val_481	2008-12-31	11
-457	val_457	2008-12-31	11
-98	val_98	2008-12-31	11
-282	val_282	2008-12-31	11
-197	val_197	2008-12-31	11
-187	val_187	2008-12-31	11
-318	val_318	2008-12-31	11
-318	val_318	2008-12-31	11
-409	val_409	2008-12-31	11
-470	val_470	2008-12-31	11
-137	val_137	2008-12-31	11
-369	val_369	2008-12-31	11
-316	val_316	2008-12-31	11
-169	val_169	2008-12-31	11
-413	val_413	2008-12-31	11
-85	val_85	2008-12-31	11
-77	val_77	2008-12-31	11
-0	val_0	2008-12-31	11
-490	val_490	2008-12-31	11
-87	val_87	2008-12-31	11
-364	val_364	2008-12-31	11
-179	val_179	2008-12-31	11
-118	val_118	2008-12-31	11
-134	val_134	2008-12-31	11
-395	val_395	2008-12-31	11
-282	val_282	2008-12-31	11
-138	val_138	2008-12-31	11
-238	val_238	2008-12-31	11
-419	val_419	2008-12-31	11
-15	val_15	2008-12-31	11
-118	val_118	2008-12-31	11
-72	val_72	2008-12-31	11
-90	val_90	2008-12-31	11
-307	val_307	2008-12-31	11
-19	val_19	2008-12-31	11
-435	val_435	2008-12-31	11
-10	val_10	2008-12-31	11
-277	val_277	2008-12-31	11
-273	val_273	2008-12-31	11
-306	val_306	2008-12-31	11
-224	val_224	2008-12-31	11
-309	val_309	2008-12-31	11
-389	val_389	2008-12-31	11
-327	val_327	2008-12-31	11
-242	val_242	2008-12-31	11
-369	val_369	2008-12-31	11
-392	val_392	2008-12-31	11
-272	val_272	2008-12-31	11
-331	val_331	2008-12-31	11
-401	val_401	2008-12-31	11
-242	val_242	2008-12-31	11
-452	val_452	2008-12-31	11
-177	val_177	2008-12-31	11
-226	val_226	2008-12-31	11
-5	val_5	2008-12-31	11
-497	val_497	2008-12-31	11
-402	val_402	2008-12-31	11
-396	val_396	2008-12-31	11
-317	val_317	2008-12-31	11
-395	val_395	2008-12-31	11
-58	val_58	2008-12-31	11
-35	val_35	2008-12-31	11
-336	val_336	2008-12-31	11
-95	val_95	2008-12-31	11
-11	val_11	2008-12-31	11
-168	val_168	2008-12-31	11
-34	val_34	2008-12-31	11
-229	val_229	2008-12-31	11
-233	val_233	2008-12-31	11
-143	val_143	2008-12-31	11
-472	val_472	2008-12-31	11
-322	val_322	2008-12-31	11
-498	val_498	2008-12-31	11
-160	val_160	2008-12-31	11
-195	val_195	2008-12-31	11
-42	val_42	2008-12-31	11
-321	val_321	2008-12-31	11
-430	val_430	2008-12-31	11
-119	val_119	2008-12-31	11
-489	val_489	2008-12-31	11
-458	val_458	2008-12-31	11
-78	val_78	2008-12-31	11
-76	val_76	2008-12-31	11
-41	val_41	2008-12-31	11
-223	val_223	2008-12-31	11
-492	val_492	2008-12-31	11
-149	val_149	2008-12-31	11
-449	val_449	2008-12-31	11
-218	val_218	2008-12-31	11
-228	val_228	2008-12-31	11
-138	val_138	2008-12-31	11
-453	val_453	2008-12-31	11
-30	val_30	2008-12-31	11
-209	val_209	2008-12-31	11
-64	val_64	2008-12-31	11
-468	val_468	2008-12-31	11
-76	val_76	2008-12-31	11
-74	val_74	2008-12-31	11
-342	val_342	2008-12-31	11
-69	val_69	2008-12-31	11
-230	val_230	2008-12-31	11
-33	val_33	2008-12-31	11
-368	val_368	2008-12-31	11
-103	val_103	2008-12-31	11
-296	val_296	2008-12-31	11
-113	val_113	2008-12-31	11
-216	val_216	2008-12-31	11
-367	val_367	2008-12-31	11
-344	val_344	2008-12-31	11
-167	val_167	2008-12-31	11
-274	val_274	2008-12-31	11
-219	val_219	2008-12-31	11
-239	val_239	2008-12-31	11
-485	val_485	2008-12-31	11
-116	val_116	2008-12-31	11
-223	val_223	2008-12-31	11
-256	val_256	2008-12-31	11
-263	val_263	2008-12-31	11
-70	val_70	2008-12-31	11
-487	val_487	2008-12-31	11
-480	val_480	2008-12-31	11
-401	val_401	2008-12-31	11
-288	val_288	2008-12-31	11
-191	val_191	2008-12-31	11
-5	val_5	2008-12-31	11
-244	val_244	2008-12-31	11
-438	val_438	2008-12-31	11
-128	val_128	2008-12-31	11
-467	val_467	2008-12-31	11
-432	val_432	2008-12-31	11
-202	val_202	2008-12-31	11
-316	val_316	2008-12-31	11
-229	val_229	2008-12-31	11
-469	val_469	2008-12-31	11
-463	val_463	2008-12-31	11
-280	val_280	2008-12-31	11
-2	val_2	2008-12-31	11
-35	val_35	2008-12-31	11
-283	val_283	2008-12-31	11
-331	val_331	2008-12-31	11
-235	val_235	2008-12-31	11
-80	val_80	2008-12-31	11
-44	val_44	2008-12-31	11
-193	val_193	2008-12-31	11
-321	val_321	2008-12-31	11
-335	val_335	2008-12-31	11
-104	val_104	2008-12-31	11
-466	val_466	2008-12-31	11
-366	val_366	2008-12-31	11
-175	val_175	2008-12-31	11
-403	val_403	2008-12-31	11
-483	val_483	2008-12-31	11
-53	val_53	2008-12-31	11
-105	val_105	2008-12-31	11
-257	val_257	2008-12-31	11
-406	val_406	2008-12-31	11
-409	val_409	2008-12-31	11
-190	val_190	2008-12-31	11
-406	val_406	2008-12-31	11
-401	val_401	2008-12-31	11
-114	val_114	2008-12-31	11
-258	val_258	2008-12-31	11
-90	val_90	2008-12-31	11
-203	val_203	2008-12-31	11
-262	val_262	2008-12-31	11
-348	val_348	2008-12-31	11
-424	val_424	2008-12-31	11
-12	val_12	2008-12-31	11
-396	val_396	2008-12-31	11
-201	val_201	2008-12-31	11
-217	val_217	2008-12-31	11
-164	val_164	2008-12-31	11
-431	val_431	2008-12-31	11
-454	val_454	2008-12-31	11
-478	val_478	2008-12-31	11
-298	val_298	2008-12-31	11
-125	val_125	2008-12-31	11
-431	val_431	2008-12-31	11
-164	val_164	2008-12-31	11
-424	val_424	2008-12-31	11
-187	val_187	2008-12-31	11
-382	val_382	2008-12-31	11
-5	val_5	2008-12-31	11
-70	val_70	2008-12-31	11
-397	val_397	2008-12-31	11
-480	val_480	2008-12-31	11
-291	val_291	2008-12-31	11
-24	val_24	2008-12-31	11
-351	val_351	2008-12-31	11
-255	val_255	2008-12-31	11
-104	val_104	2008-12-31	11
-70	val_70	2008-12-31	11
-163	val_163	2008-12-31	11
-438	val_438	2008-12-31	11
-119	val_119	2008-12-31	11
-414	val_414	2008-12-31	11
-200	val_200	2008-12-31	11
-491	val_491	2008-12-31	11
-237	val_237	2008-12-31	11
-439	val_439	2008-12-31	11
-360	val_360	2008-12-31	11
-248	val_248	2008-12-31	11
-479	val_479	2008-12-31	11
-305	val_305	2008-12-31	11
-417	val_417	2008-12-31	11
-199	val_199	2008-12-31	11
-444	val_444	2008-12-31	11
-120	val_120	2008-12-31	11
-429	val_429	2008-12-31	11
-169	val_169	2008-12-31	11
-443	val_443	2008-12-31	11
-323	val_323	2008-12-31	11
-325	val_325	2008-12-31	11
-277	val_277	2008-12-31	11
-230	val_230	2008-12-31	11
-478	val_478	2008-12-31	11
-178	val_178	2008-12-31	11
-468	val_468	2008-12-31	11
-310	val_310	2008-12-31	11
-317	val_317	2008-12-31	11
-333	val_333	2008-12-31	11
-493	val_493	2008-12-31	11
-460	val_460	2008-12-31	11
-207	val_207	2008-12-31	11
-249	val_249	2008-12-31	11
-265	val_265	2008-12-31	11
-480	val_480	2008-12-31	11
-83	val_83	2008-12-31	11
-136	val_136	2008-12-31	11
-353	val_353	2008-12-31	11
-172	val_172	2008-12-31	11
-214	val_214	2008-12-31	11
-462	val_462	2008-12-31	11
-233	val_233	2008-12-31	11
-406	val_406	2008-12-31	11
-133	val_133	2008-12-31	11
-175	val_175	2008-12-31	11
-189	val_189	2008-12-31	11
-454	val_454	2008-12-31	11
-375	val_375	2008-12-31	11
-401	val_401	2008-12-31	11
-421	val_421	2008-12-31	11
-407	val_407	2008-12-31	11
-384	val_384	2008-12-31	11
-256	val_256	2008-12-31	11
-26	val_26	2008-12-31	11
-134	val_134	2008-12-31	11
-67	val_67	2008-12-31	11
-384	val_384	2008-12-31	11
-379	val_379	2008-12-31	11
-18	val_18	2008-12-31	11
-462	val_462	2008-12-31	11
-492	val_492	2008-12-31	11
-100	val_100	2008-12-31	11
-298	val_298	2008-12-31	11
-9	val_9	2008-12-31	11
-341	val_341	2008-12-31	11
-498	val_498	2008-12-31	11
-146	val_146	2008-12-31	11
-458	val_458	2008-12-31	11
-362	val_362	2008-12-31	11
-186	val_186	2008-12-31	11
-285	val_285	2008-12-31	11
-348	val_348	2008-12-31	11
-167	val_167	2008-12-31	11
-18	val_18	2008-12-31	11
-273	val_273	2008-12-31	11
-183	val_183	2008-12-31	11
-281	val_281	2008-12-31	11
-344	val_344	2008-12-31	11
-97	val_97	2008-12-31	11
-469	val_469	2008-12-31	11
-315	val_315	2008-12-31	11
-84	val_84	2008-12-31	11
-28	val_28	2008-12-31	11
-37	val_37	2008-12-31	11
-448	val_448	2008-12-31	11
-152	val_152	2008-12-31	11
-348	val_348	2008-12-31	11
-307	val_307	2008-12-31	11
-194	val_194	2008-12-31	11
-414	val_414	2008-12-31	11
-477	val_477	2008-12-31	11
-222	val_222	2008-12-31	11
-126	val_126	2008-12-31	11
-90	val_90	2008-12-31	11
-169	val_169	2008-12-31	11
-403	val_403	2008-12-31	11
-400	val_400	2008-12-31	11
-200	val_200	2008-12-31	11
-97	val_97	2008-12-31	11
-238	val_238	2008-12-31	12
-86	val_86	2008-12-31	12
-311	val_311	2008-12-31	12
-27	val_27	2008-12-31	12
-165	val_165	2008-12-31	12
-409	val_409	2008-12-31	12
-255	val_255	2008-12-31	12
-278	val_278	2008-12-31	12
-98	val_98	2008-12-31	12
-484	val_484	2008-12-31	12
-265	val_265	2008-12-31	12
-193	val_193	2008-12-31	12
-401	val_401	2008-12-31	12
-150	val_150	2008-12-31	12
-273	val_273	2008-12-31	12
-224	val_224	2008-12-31	12
-369	val_369	2008-12-31	12
-66	val_66	2008-12-31	12
-128	val_128	2008-12-31	12
-213	val_213	2008-12-31	12
-146	val_146	2008-12-31	12
-406	val_406	2008-12-31	12
-429	val_429	2008-12-31	12
-374	val_374	2008-12-31	12
-152	val_152	2008-12-31	12
-469	val_469	2008-12-31	12
-145	val_145	2008-12-31	12
-495	val_495	2008-12-31	12
-37	val_37	2008-12-31	12
-327	val_327	2008-12-31	12
-281	val_281	2008-12-31	12
-277	val_277	2008-12-31	12
-209	val_209	2008-12-31	12
-15	val_15	2008-12-31	12
-82	val_82	2008-12-31	12
-403	val_403	2008-12-31	12
-166	val_166	2008-12-31	12
-417	val_417	2008-12-31	12
-430	val_430	2008-12-31	12
-252	val_252	2008-12-31	12
-292	val_292	2008-12-31	12
-219	val_219	2008-12-31	12
-287	val_287	2008-12-31	12
-153	val_153	2008-12-31	12
-193	val_193	2008-12-31	12
-338	val_338	2008-12-31	12
-446	val_446	2008-12-31	12
-459	val_459	2008-12-31	12
-394	val_394	2008-12-31	12
-237	val_237	2008-12-31	12
-482	val_482	2008-12-31	12
-174	val_174	2008-12-31	12
-413	val_413	2008-12-31	12
-494	val_494	2008-12-31	12
-207	val_207	2008-12-31	12
-199	val_199	2008-12-31	12
-466	val_466	2008-12-31	12
-208	val_208	2008-12-31	12
-174	val_174	2008-12-31	12
-399	val_399	2008-12-31	12
-396	val_396	2008-12-31	12
-247	val_247	2008-12-31	12
-417	val_417	2008-12-31	12
-489	val_489	2008-12-31	12
-162	val_162	2008-12-31	12
-377	val_377	2008-12-31	12
-397	val_397	2008-12-31	12
-309	val_309	2008-12-31	12
-365	val_365	2008-12-31	12
-266	val_266	2008-12-31	12
-439	val_439	2008-12-31	12
-342	val_342	2008-12-31	12
-367	val_367	2008-12-31	12
-325	val_325	2008-12-31	12
-167	val_167	2008-12-31	12
-195	val_195	2008-12-31	12
-475	val_475	2008-12-31	12
-17	val_17	2008-12-31	12
-113	val_113	2008-12-31	12
-155	val_155	2008-12-31	12
-203	val_203	2008-12-31	12
-339	val_339	2008-12-31	12
-0	val_0	2008-12-31	12
-455	val_455	2008-12-31	12
-128	val_128	2008-12-31	12
-311	val_311	2008-12-31	12
-316	val_316	2008-12-31	12
-57	val_57	2008-12-31	12
-302	val_302	2008-12-31	12
-205	val_205	2008-12-31	12
-149	val_149	2008-12-31	12
-438	val_438	2008-12-31	12
-345	val_345	2008-12-31	12
-129	val_129	2008-12-31	12
-170	val_170	2008-12-31	12
-20	val_20	2008-12-31	12
-489	val_489	2008-12-31	12
-157	val_157	2008-12-31	12
-378	val_378	2008-12-31	12
-221	val_221	2008-12-31	12
-92	val_92	2008-12-31	12
-111	val_111	2008-12-31	12
-47	val_47	2008-12-31	12
-72	val_72	2008-12-31	12
-4	val_4	2008-12-31	12
-280	val_280	2008-12-31	12
-35	val_35	2008-12-31	12
-427	val_427	2008-12-31	12
-277	val_277	2008-12-31	12
-208	val_208	2008-12-31	12
-356	val_356	2008-12-31	12
-399	val_399	2008-12-31	12
-169	val_169	2008-12-31	12
-382	val_382	2008-12-31	12
-498	val_498	2008-12-31	12
-125	val_125	2008-12-31	12
-386	val_386	2008-12-31	12
-437	val_437	2008-12-31	12
-469	val_469	2008-12-31	12
-192	val_192	2008-12-31	12
-286	val_286	2008-12-31	12
-187	val_187	2008-12-31	12
-176	val_176	2008-12-31	12
-54	val_54	2008-12-31	12
-459	val_459	2008-12-31	12
-51	val_51	2008-12-31	12
-138	val_138	2008-12-31	12
-103	val_103	2008-12-31	12
-239	val_239	2008-12-31	12
-213	val_213	2008-12-31	12
-216	val_216	2008-12-31	12
-430	val_430	2008-12-31	12
-278	val_278	2008-12-31	12
-176	val_176	2008-12-31	12
-289	val_289	2008-12-31	12
-221	val_221	2008-12-31	12
-65	val_65	2008-12-31	12
-318	val_318	2008-12-31	12
-332	val_332	2008-12-31	12
-311	val_311	2008-12-31	12
-275	val_275	2008-12-31	12
-137	val_137	2008-12-31	12
-241	val_241	2008-12-31	12
-83	val_83	2008-12-31	12
-333	val_333	2008-12-31	12
-180	val_180	2008-12-31	12
-284	val_284	2008-12-31	12
-12	val_12	2008-12-31	12
-230	val_230	2008-12-31	12
-181	val_181	2008-12-31	12
-67	val_67	2008-12-31	12
-260	val_260	2008-12-31	12
-404	val_404	2008-12-31	12
-384	val_384	2008-12-31	12
-489	val_489	2008-12-31	12
-353	val_353	2008-12-31	12
-373	val_373	2008-12-31	12
-272	val_272	2008-12-31	12
-138	val_138	2008-12-31	12
-217	val_217	2008-12-31	12
-84	val_84	2008-12-31	12
-348	val_348	2008-12-31	12
-466	val_466	2008-12-31	12
-58	val_58	2008-12-31	12
-8	val_8	2008-12-31	12
-411	val_411	2008-12-31	12
-230	val_230	2008-12-31	12
-208	val_208	2008-12-31	12
-348	val_348	2008-12-31	12
-24	val_24	2008-12-31	12
-463	val_463	2008-12-31	12
-431	val_431	2008-12-31	12
-179	val_179	2008-12-31	12
-172	val_172	2008-12-31	12
-42	val_42	2008-12-31	12
-129	val_129	2008-12-31	12
-158	val_158	2008-12-31	12
-119	val_119	2008-12-31	12
-496	val_496	2008-12-31	12
-0	val_0	2008-12-31	12
-322	val_322	2008-12-31	12
-197	val_197	2008-12-31	12
-468	val_468	2008-12-31	12
-393	val_393	2008-12-31	12
-454	val_454	2008-12-31	12
-100	val_100	2008-12-31	12
-298	val_298	2008-12-31	12
-199	val_199	2008-12-31	12
-191	val_191	2008-12-31	12
-418	val_418	2008-12-31	12
-96	val_96	2008-12-31	12
-26	val_26	2008-12-31	12
-165	val_165	2008-12-31	12
-327	val_327	2008-12-31	12
-230	val_230	2008-12-31	12
-205	val_205	2008-12-31	12
-120	val_120	2008-12-31	12
-131	val_131	2008-12-31	12
-51	val_51	2008-12-31	12
-404	val_404	2008-12-31	12
-43	val_43	2008-12-31	12
-436	val_436	2008-12-31	12
-156	val_156	2008-12-31	12
-469	val_469	2008-12-31	12
-468	val_468	2008-12-31	12
-308	val_308	2008-12-31	12
-95	val_95	2008-12-31	12
-196	val_196	2008-12-31	12
-288	val_288	2008-12-31	12
-481	val_481	2008-12-31	12
-457	val_457	2008-12-31	12
-98	val_98	2008-12-31	12
-282	val_282	2008-12-31	12
-197	val_197	2008-12-31	12
-187	val_187	2008-12-31	12
-318	val_318	2008-12-31	12
-318	val_318	2008-12-31	12
-409	val_409	2008-12-31	12
-470	val_470	2008-12-31	12
-137	val_137	2008-12-31	12
-369	val_369	2008-12-31	12
-316	val_316	2008-12-31	12
-169	val_169	2008-12-31	12
-413	val_413	2008-12-31	12
-85	val_85	2008-12-31	12
-77	val_77	2008-12-31	12
-0	val_0	2008-12-31	12
-490	val_490	2008-12-31	12
-87	val_87	2008-12-31	12
-364	val_364	2008-12-31	12
-179	val_179	2008-12-31	12
-118	val_118	2008-12-31	12
-134	val_134	2008-12-31	12
-395	val_395	2008-12-31	12
-282	val_282	2008-12-31	12
-138	val_138	2008-12-31	12
-238	val_238	2008-12-31	12
-419	val_419	2008-12-31	12
-15	val_15	2008-12-31	12
-118	val_118	2008-12-31	12
-72	val_72	2008-12-31	12
-90	val_90	2008-12-31	12
-307	val_307	2008-12-31	12
-19	val_19	2008-12-31	12
-435	val_435	2008-12-31	12
-10	val_10	2008-12-31	12
-277	val_277	2008-12-31	12
-273	val_273	2008-12-31	12
-306	val_306	2008-12-31	12
-224	val_224	2008-12-31	12
-309	val_309	2008-12-31	12
-389	val_389	2008-12-31	12
-327	val_327	2008-12-31	12
-242	val_242	2008-12-31	12
-369	val_369	2008-12-31	12
-392	val_392	2008-12-31	12
-272	val_272	2008-12-31	12
-331	val_331	2008-12-31	12
-401	val_401	2008-12-31	12
-242	val_242	2008-12-31	12
-452	val_452	2008-12-31	12
-177	val_177	2008-12-31	12
-226	val_226	2008-12-31	12
-5	val_5	2008-12-31	12
-497	val_497	2008-12-31	12
-402	val_402	2008-12-31	12
-396	val_396	2008-12-31	12
-317	val_317	2008-12-31	12
-395	val_395	2008-12-31	12
-58	val_58	2008-12-31	12
-35	val_35	2008-12-31	12
-336	val_336	2008-12-31	12
-95	val_95	2008-12-31	12
-11	val_11	2008-12-31	12
-168	val_168	2008-12-31	12
-34	val_34	2008-12-31	12
-229	val_229	2008-12-31	12
-233	val_233	2008-12-31	12
-143	val_143	2008-12-31	12
-472	val_472	2008-12-31	12
-322	val_322	2008-12-31	12
-498	val_498	2008-12-31	12
-160	val_160	2008-12-31	12
-195	val_195	2008-12-31	12
-42	val_42	2008-12-31	12
-321	val_321	2008-12-31	12
-430	val_430	2008-12-31	12
-119	val_119	2008-12-31	12
-489	val_489	2008-12-31	12
-458	val_458	2008-12-31	12
-78	val_78	2008-12-31	12
-76	val_76	2008-12-31	12
-41	val_41	2008-12-31	12
-223	val_223	2008-12-31	12
-492	val_492	2008-12-31	12
-149	val_149	2008-12-31	12
-449	val_449	2008-12-31	12
-218	val_218	2008-12-31	12
-228	val_228	2008-12-31	12
-138	val_138	2008-12-31	12
-453	val_453	2008-12-31	12
-30	val_30	2008-12-31	12
-209	val_209	2008-12-31	12
-64	val_64	2008-12-31	12
-468	val_468	2008-12-31	12
-76	val_76	2008-12-31	12
-74	val_74	2008-12-31	12
-342	val_342	2008-12-31	12
-69	val_69	2008-12-31	12
-230	val_230	2008-12-31	12
-33	val_33	2008-12-31	12
-368	val_368	2008-12-31	12
-103	val_103	2008-12-31	12
-296	val_296	2008-12-31	12
-113	val_113	2008-12-31	12
-216	val_216	2008-12-31	12
-367	val_367	2008-12-31	12
-344	val_344	2008-12-31	12
-167	val_167	2008-12-31	12
-274	val_274	2008-12-31	12
-219	val_219	2008-12-31	12
-239	val_239	2008-12-31	12
-485	val_485	2008-12-31	12
-116	val_116	2008-12-31	12
-223	val_223	2008-12-31	12
-256	val_256	2008-12-31	12
-263	val_263	2008-12-31	12
-70	val_70	2008-12-31	12
-487	val_487	2008-12-31	12
-480	val_480	2008-12-31	12
-401	val_401	2008-12-31	12
-288	val_288	2008-12-31	12
-191	val_191	2008-12-31	12
-5	val_5	2008-12-31	12
-244	val_244	2008-12-31	12
-438	val_438	2008-12-31	12
-128	val_128	2008-12-31	12
-467	val_467	2008-12-31	12
-432	val_432	2008-12-31	12
-202	val_202	2008-12-31	12
-316	val_316	2008-12-31	12
-229	val_229	2008-12-31	12
-469	val_469	2008-12-31	12
-463	val_463	2008-12-31	12
-280	val_280	2008-12-31	12
-2	val_2	2008-12-31	12
-35	val_35	2008-12-31	12
-283	val_283	2008-12-31	12
-331	val_331	2008-12-31	12
-235	val_235	2008-12-31	12
-80	val_80	2008-12-31	12
-44	val_44	2008-12-31	12
-193	val_193	2008-12-31	12
-321	val_321	2008-12-31	12
-335	val_335	2008-12-31	12
-104	val_104	2008-12-31	12
-466	val_466	2008-12-31	12
-366	val_366	2008-12-31	12
-175	val_175	2008-12-31	12
-403	val_403	2008-12-31	12
-483	val_483	2008-12-31	12
-53	val_53	2008-12-31	12
-105	val_105	2008-12-31	12
-257	val_257	2008-12-31	12
-406	val_406	2008-12-31	12
-409	val_409	2008-12-31	12
-190	val_190	2008-12-31	12
-406	val_406	2008-12-31	12
-401	val_401	2008-12-31	12
-114	val_114	2008-12-31	12
-258	val_258	2008-12-31	12
-90	val_90	2008-12-31	12
-203	val_203	2008-12-31	12
-262	val_262	2008-12-31	12
-348	val_348	2008-12-31	12
-424	val_424	2008-12-31	12
-12	val_12	2008-12-31	12
-396	val_396	2008-12-31	12
-201	val_201	2008-12-31	12
-217	val_217	2008-12-31	12
-164	val_164	2008-12-31	12
-431	val_431	2008-12-31	12
-454	val_454	2008-12-31	12
-478	val_478	2008-12-31	12
-298	val_298	2008-12-31	12
-125	val_125	2008-12-31	12
-431	val_431	2008-12-31	12
-164	val_164	2008-12-31	12
-424	val_424	2008-12-31	12
-187	val_187	2008-12-31	12
-382	val_382	2008-12-31	12
-5	val_5	2008-12-31	12
-70	val_70	2008-12-31	12
-397	val_397	2008-12-31	12
-480	val_480	2008-12-31	12
-291	val_291	2008-12-31	12
-24	val_24	2008-12-31	12
-351	val_351	2008-12-31	12
-255	val_255	2008-12-31	12
-104	val_104	2008-12-31	12
-70	val_70	2008-12-31	12
-163	val_163	2008-12-31	12
-438	val_438	2008-12-31	12
-119	val_119	2008-12-31	12
-414	val_414	2008-12-31	12
-200	val_200	2008-12-31	12
-491	val_491	2008-12-31	12
-237	val_237	2008-12-31	12
-439	val_439	2008-12-31	12
-360	val_360	2008-12-31	12
-248	val_248	2008-12-31	12
-479	val_479	2008-12-31	12
-305	val_305	2008-12-31	12
-417	val_417	2008-12-31	12
-199	val_199	2008-12-31	12
-444	val_444	2008-12-31	12
-120	val_120	2008-12-31	12
-429	val_429	2008-12-31	12
-169	val_169	2008-12-31	12
-443	val_443	2008-12-31	12
-323	val_323	2008-12-31	12
-325	val_325	2008-12-31	12
-277	val_277	2008-12-31	12
-230	val_230	2008-12-31	12
-478	val_478	2008-12-31	12
-178	val_178	2008-12-31	12
-468	val_468	2008-12-31	12
-310	val_310	2008-12-31	12
-317	val_317	2008-12-31	12
-333	val_333	2008-12-31	12
-493	val_493	2008-12-31	12
-460	val_460	2008-12-31	12
-207	val_207	2008-12-31	12
-249	val_249	2008-12-31	12
-265	val_265	2008-12-31	12
-480	val_480	2008-12-31	12
-83	val_83	2008-12-31	12
-136	val_136	2008-12-31	12
-353	val_353	2008-12-31	12
-172	val_172	2008-12-31	12
-214	val_214	2008-12-31	12
-462	val_462	2008-12-31	12
-233	val_233	2008-12-31	12
-406	val_406	2008-12-31	12
-133	val_133	2008-12-31	12
-175	val_175	2008-12-31	12
-189	val_189	2008-12-31	12
-454	val_454	2008-12-31	12
-375	val_375	2008-12-31	12
-401	val_401	2008-12-31	12
-421	val_421	2008-12-31	12
-407	val_407	2008-12-31	12
-384	val_384	2008-12-31	12
-256	val_256	2008-12-31	12
-26	val_26	2008-12-31	12
-134	val_134	2008-12-31	12
-67	val_67	2008-12-31	12
-384	val_384	2008-12-31	12
-379	val_379	2008-12-31	12
-18	val_18	2008-12-31	12
-462	val_462	2008-12-31	12
-492	val_492	2008-12-31	12
-100	val_100	2008-12-31	12
-298	val_298	2008-12-31	12
-9	val_9	2008-12-31	12
-341	val_341	2008-12-31	12
-498	val_498	2008-12-31	12
-146	val_146	2008-12-31	12
-458	val_458	2008-12-31	12
-362	val_362	2008-12-31	12
-186	val_186	2008-12-31	12
-285	val_285	2008-12-31	12
-348	val_348	2008-12-31	12
-167	val_167	2008-12-31	12
-18	val_18	2008-12-31	12
-273	val_273	2008-12-31	12
-183	val_183	2008-12-31	12
-281	val_281	2008-12-31	12
-344	val_344	2008-12-31	12
-97	val_97	2008-12-31	12
-469	val_469	2008-12-31	12
-315	val_315	2008-12-31	12
-84	val_84	2008-12-31	12
-28	val_28	2008-12-31	12
-37	val_37	2008-12-31	12
-448	val_448	2008-12-31	12
-152	val_152	2008-12-31	12
-348	val_348	2008-12-31	12
-307	val_307	2008-12-31	12
-194	val_194	2008-12-31	12
-414	val_414	2008-12-31	12
-477	val_477	2008-12-31	12
-222	val_222	2008-12-31	12
-126	val_126	2008-12-31	12
-90	val_90	2008-12-31	12
-169	val_169	2008-12-31	12
-403	val_403	2008-12-31	12
-400	val_400	2008-12-31	12
-200	val_200	2008-12-31	12
-97	val_97	2008-12-31	12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a b/sql/hive/src/test/resources/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a
index 267cb634d6bd33cbfa0716219cae4c69dfbba0d4..1240429107ddbf00a3f0278e9f860bccbf2e38b9 100644
--- a/sql/hive/src/test/resources/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a
+++ b/sql/hive/src/test/resources/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a
@@ -1,12 +1,12 @@
-key                 	int                 	None                
-value               	string              	None                
-ds                  	string              	None                
-hr                  	string              	None                
+key                 	int                 	                    
+value               	string              	                    
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
-hr                  	string              	None                
+ds                  	string              	                    
+hr                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:nzhang_part9, dbName:default, owner:marmbrus, createTime:1390899685, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part9, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{p3=v3, transient_lastDdlTime=1390899685}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:nzhang_part9, dbName:default, owner:marmbrus, createTime:1413887729, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/nzhang_part9, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], parameters:{transient_lastDdlTime=1413887729}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part9-9-6954f5fc8dd82ca2c076ab8bcdbc148 b/sql/hive/src/test/resources/golden/load_dyn_part9-9-6954f5fc8dd82ca2c076ab8bcdbc148
new file mode 100644
index 0000000000000000000000000000000000000000..04b36182974f919450daa1dacd7e2371b61bdb51
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/load_dyn_part9-9-6954f5fc8dd82ca2c076ab8bcdbc148
@@ -0,0 +1,1000 @@
+0	val_0	2008-04-08	11
+0	val_0	2008-04-08	11
+0	val_0	2008-04-08	11
+2	val_2	2008-04-08	11
+4	val_4	2008-04-08	11
+5	val_5	2008-04-08	11
+5	val_5	2008-04-08	11
+5	val_5	2008-04-08	11
+8	val_8	2008-04-08	11
+9	val_9	2008-04-08	11
+10	val_10	2008-04-08	11
+11	val_11	2008-04-08	11
+12	val_12	2008-04-08	11
+12	val_12	2008-04-08	11
+15	val_15	2008-04-08	11
+15	val_15	2008-04-08	11
+17	val_17	2008-04-08	11
+18	val_18	2008-04-08	11
+18	val_18	2008-04-08	11
+19	val_19	2008-04-08	11
+20	val_20	2008-04-08	11
+24	val_24	2008-04-08	11
+24	val_24	2008-04-08	11
+26	val_26	2008-04-08	11
+26	val_26	2008-04-08	11
+27	val_27	2008-04-08	11
+28	val_28	2008-04-08	11
+30	val_30	2008-04-08	11
+33	val_33	2008-04-08	11
+34	val_34	2008-04-08	11
+35	val_35	2008-04-08	11
+35	val_35	2008-04-08	11
+35	val_35	2008-04-08	11
+37	val_37	2008-04-08	11
+37	val_37	2008-04-08	11
+41	val_41	2008-04-08	11
+42	val_42	2008-04-08	11
+42	val_42	2008-04-08	11
+43	val_43	2008-04-08	11
+44	val_44	2008-04-08	11
+47	val_47	2008-04-08	11
+51	val_51	2008-04-08	11
+51	val_51	2008-04-08	11
+53	val_53	2008-04-08	11
+54	val_54	2008-04-08	11
+57	val_57	2008-04-08	11
+58	val_58	2008-04-08	11
+58	val_58	2008-04-08	11
+64	val_64	2008-04-08	11
+65	val_65	2008-04-08	11
+66	val_66	2008-04-08	11
+67	val_67	2008-04-08	11
+67	val_67	2008-04-08	11
+69	val_69	2008-04-08	11
+70	val_70	2008-04-08	11
+70	val_70	2008-04-08	11
+70	val_70	2008-04-08	11
+72	val_72	2008-04-08	11
+72	val_72	2008-04-08	11
+74	val_74	2008-04-08	11
+76	val_76	2008-04-08	11
+76	val_76	2008-04-08	11
+77	val_77	2008-04-08	11
+78	val_78	2008-04-08	11
+80	val_80	2008-04-08	11
+82	val_82	2008-04-08	11
+83	val_83	2008-04-08	11
+83	val_83	2008-04-08	11
+84	val_84	2008-04-08	11
+84	val_84	2008-04-08	11
+85	val_85	2008-04-08	11
+86	val_86	2008-04-08	11
+87	val_87	2008-04-08	11
+90	val_90	2008-04-08	11
+90	val_90	2008-04-08	11
+90	val_90	2008-04-08	11
+92	val_92	2008-04-08	11
+95	val_95	2008-04-08	11
+95	val_95	2008-04-08	11
+96	val_96	2008-04-08	11
+97	val_97	2008-04-08	11
+97	val_97	2008-04-08	11
+98	val_98	2008-04-08	11
+98	val_98	2008-04-08	11
+100	val_100	2008-04-08	11
+100	val_100	2008-04-08	11
+103	val_103	2008-04-08	11
+103	val_103	2008-04-08	11
+104	val_104	2008-04-08	11
+104	val_104	2008-04-08	11
+105	val_105	2008-04-08	11
+111	val_111	2008-04-08	11
+113	val_113	2008-04-08	11
+113	val_113	2008-04-08	11
+114	val_114	2008-04-08	11
+116	val_116	2008-04-08	11
+118	val_118	2008-04-08	11
+118	val_118	2008-04-08	11
+119	val_119	2008-04-08	11
+119	val_119	2008-04-08	11
+119	val_119	2008-04-08	11
+120	val_120	2008-04-08	11
+120	val_120	2008-04-08	11
+125	val_125	2008-04-08	11
+125	val_125	2008-04-08	11
+126	val_126	2008-04-08	11
+128	val_128	2008-04-08	11
+128	val_128	2008-04-08	11
+128	val_128	2008-04-08	11
+129	val_129	2008-04-08	11
+129	val_129	2008-04-08	11
+131	val_131	2008-04-08	11
+133	val_133	2008-04-08	11
+134	val_134	2008-04-08	11
+134	val_134	2008-04-08	11
+136	val_136	2008-04-08	11
+137	val_137	2008-04-08	11
+137	val_137	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+138	val_138	2008-04-08	11
+143	val_143	2008-04-08	11
+145	val_145	2008-04-08	11
+146	val_146	2008-04-08	11
+146	val_146	2008-04-08	11
+149	val_149	2008-04-08	11
+149	val_149	2008-04-08	11
+150	val_150	2008-04-08	11
+152	val_152	2008-04-08	11
+152	val_152	2008-04-08	11
+153	val_153	2008-04-08	11
+155	val_155	2008-04-08	11
+156	val_156	2008-04-08	11
+157	val_157	2008-04-08	11
+158	val_158	2008-04-08	11
+160	val_160	2008-04-08	11
+162	val_162	2008-04-08	11
+163	val_163	2008-04-08	11
+164	val_164	2008-04-08	11
+164	val_164	2008-04-08	11
+165	val_165	2008-04-08	11
+165	val_165	2008-04-08	11
+166	val_166	2008-04-08	11
+167	val_167	2008-04-08	11
+167	val_167	2008-04-08	11
+167	val_167	2008-04-08	11
+168	val_168	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+169	val_169	2008-04-08	11
+170	val_170	2008-04-08	11
+172	val_172	2008-04-08	11
+172	val_172	2008-04-08	11
+174	val_174	2008-04-08	11
+174	val_174	2008-04-08	11
+175	val_175	2008-04-08	11
+175	val_175	2008-04-08	11
+176	val_176	2008-04-08	11
+176	val_176	2008-04-08	11
+177	val_177	2008-04-08	11
+178	val_178	2008-04-08	11
+179	val_179	2008-04-08	11
+179	val_179	2008-04-08	11
+180	val_180	2008-04-08	11
+181	val_181	2008-04-08	11
+183	val_183	2008-04-08	11
+186	val_186	2008-04-08	11
+187	val_187	2008-04-08	11
+187	val_187	2008-04-08	11
+187	val_187	2008-04-08	11
+189	val_189	2008-04-08	11
+190	val_190	2008-04-08	11
+191	val_191	2008-04-08	11
+191	val_191	2008-04-08	11
+192	val_192	2008-04-08	11
+193	val_193	2008-04-08	11
+193	val_193	2008-04-08	11
+193	val_193	2008-04-08	11
+194	val_194	2008-04-08	11
+195	val_195	2008-04-08	11
+195	val_195	2008-04-08	11
+196	val_196	2008-04-08	11
+197	val_197	2008-04-08	11
+197	val_197	2008-04-08	11
+199	val_199	2008-04-08	11
+199	val_199	2008-04-08	11
+199	val_199	2008-04-08	11
+200	val_200	2008-04-08	11
+200	val_200	2008-04-08	11
+201	val_201	2008-04-08	11
+202	val_202	2008-04-08	11
+203	val_203	2008-04-08	11
+203	val_203	2008-04-08	11
+205	val_205	2008-04-08	11
+205	val_205	2008-04-08	11
+207	val_207	2008-04-08	11
+207	val_207	2008-04-08	11
+208	val_208	2008-04-08	11
+208	val_208	2008-04-08	11
+208	val_208	2008-04-08	11
+209	val_209	2008-04-08	11
+209	val_209	2008-04-08	11
+213	val_213	2008-04-08	11
+213	val_213	2008-04-08	11
+214	val_214	2008-04-08	11
+216	val_216	2008-04-08	11
+216	val_216	2008-04-08	11
+217	val_217	2008-04-08	11
+217	val_217	2008-04-08	11
+218	val_218	2008-04-08	11
+219	val_219	2008-04-08	11
+219	val_219	2008-04-08	11
+221	val_221	2008-04-08	11
+221	val_221	2008-04-08	11
+222	val_222	2008-04-08	11
+223	val_223	2008-04-08	11
+223	val_223	2008-04-08	11
+224	val_224	2008-04-08	11
+224	val_224	2008-04-08	11
+226	val_226	2008-04-08	11
+228	val_228	2008-04-08	11
+229	val_229	2008-04-08	11
+229	val_229	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+230	val_230	2008-04-08	11
+233	val_233	2008-04-08	11
+233	val_233	2008-04-08	11
+235	val_235	2008-04-08	11
+237	val_237	2008-04-08	11
+237	val_237	2008-04-08	11
+238	val_238	2008-04-08	11
+238	val_238	2008-04-08	11
+239	val_239	2008-04-08	11
+239	val_239	2008-04-08	11
+241	val_241	2008-04-08	11
+242	val_242	2008-04-08	11
+242	val_242	2008-04-08	11
+244	val_244	2008-04-08	11
+247	val_247	2008-04-08	11
+248	val_248	2008-04-08	11
+249	val_249	2008-04-08	11
+252	val_252	2008-04-08	11
+255	val_255	2008-04-08	11
+255	val_255	2008-04-08	11
+256	val_256	2008-04-08	11
+256	val_256	2008-04-08	11
+257	val_257	2008-04-08	11
+258	val_258	2008-04-08	11
+260	val_260	2008-04-08	11
+262	val_262	2008-04-08	11
+263	val_263	2008-04-08	11
+265	val_265	2008-04-08	11
+265	val_265	2008-04-08	11
+266	val_266	2008-04-08	11
+272	val_272	2008-04-08	11
+272	val_272	2008-04-08	11
+273	val_273	2008-04-08	11
+273	val_273	2008-04-08	11
+273	val_273	2008-04-08	11
+274	val_274	2008-04-08	11
+275	val_275	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+277	val_277	2008-04-08	11
+278	val_278	2008-04-08	11
+278	val_278	2008-04-08	11
+280	val_280	2008-04-08	11
+280	val_280	2008-04-08	11
+281	val_281	2008-04-08	11
+281	val_281	2008-04-08	11
+282	val_282	2008-04-08	11
+282	val_282	2008-04-08	11
+283	val_283	2008-04-08	11
+284	val_284	2008-04-08	11
+285	val_285	2008-04-08	11
+286	val_286	2008-04-08	11
+287	val_287	2008-04-08	11
+288	val_288	2008-04-08	11
+288	val_288	2008-04-08	11
+289	val_289	2008-04-08	11
+291	val_291	2008-04-08	11
+292	val_292	2008-04-08	11
+296	val_296	2008-04-08	11
+298	val_298	2008-04-08	11
+298	val_298	2008-04-08	11
+298	val_298	2008-04-08	11
+302	val_302	2008-04-08	11
+305	val_305	2008-04-08	11
+306	val_306	2008-04-08	11
+307	val_307	2008-04-08	11
+307	val_307	2008-04-08	11
+308	val_308	2008-04-08	11
+309	val_309	2008-04-08	11
+309	val_309	2008-04-08	11
+310	val_310	2008-04-08	11
+311	val_311	2008-04-08	11
+311	val_311	2008-04-08	11
+311	val_311	2008-04-08	11
+315	val_315	2008-04-08	11
+316	val_316	2008-04-08	11
+316	val_316	2008-04-08	11
+316	val_316	2008-04-08	11
+317	val_317	2008-04-08	11
+317	val_317	2008-04-08	11
+318	val_318	2008-04-08	11
+318	val_318	2008-04-08	11
+318	val_318	2008-04-08	11
+321	val_321	2008-04-08	11
+321	val_321	2008-04-08	11
+322	val_322	2008-04-08	11
+322	val_322	2008-04-08	11
+323	val_323	2008-04-08	11
+325	val_325	2008-04-08	11
+325	val_325	2008-04-08	11
+327	val_327	2008-04-08	11
+327	val_327	2008-04-08	11
+327	val_327	2008-04-08	11
+331	val_331	2008-04-08	11
+331	val_331	2008-04-08	11
+332	val_332	2008-04-08	11
+333	val_333	2008-04-08	11
+333	val_333	2008-04-08	11
+335	val_335	2008-04-08	11
+336	val_336	2008-04-08	11
+338	val_338	2008-04-08	11
+339	val_339	2008-04-08	11
+341	val_341	2008-04-08	11
+342	val_342	2008-04-08	11
+342	val_342	2008-04-08	11
+344	val_344	2008-04-08	11
+344	val_344	2008-04-08	11
+345	val_345	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+348	val_348	2008-04-08	11
+351	val_351	2008-04-08	11
+353	val_353	2008-04-08	11
+353	val_353	2008-04-08	11
+356	val_356	2008-04-08	11
+360	val_360	2008-04-08	11
+362	val_362	2008-04-08	11
+364	val_364	2008-04-08	11
+365	val_365	2008-04-08	11
+366	val_366	2008-04-08	11
+367	val_367	2008-04-08	11
+367	val_367	2008-04-08	11
+368	val_368	2008-04-08	11
+369	val_369	2008-04-08	11
+369	val_369	2008-04-08	11
+369	val_369	2008-04-08	11
+373	val_373	2008-04-08	11
+374	val_374	2008-04-08	11
+375	val_375	2008-04-08	11
+377	val_377	2008-04-08	11
+378	val_378	2008-04-08	11
+379	val_379	2008-04-08	11
+382	val_382	2008-04-08	11
+382	val_382	2008-04-08	11
+384	val_384	2008-04-08	11
+384	val_384	2008-04-08	11
+384	val_384	2008-04-08	11
+386	val_386	2008-04-08	11
+389	val_389	2008-04-08	11
+392	val_392	2008-04-08	11
+393	val_393	2008-04-08	11
+394	val_394	2008-04-08	11
+395	val_395	2008-04-08	11
+395	val_395	2008-04-08	11
+396	val_396	2008-04-08	11
+396	val_396	2008-04-08	11
+396	val_396	2008-04-08	11
+397	val_397	2008-04-08	11
+397	val_397	2008-04-08	11
+399	val_399	2008-04-08	11
+399	val_399	2008-04-08	11
+400	val_400	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+401	val_401	2008-04-08	11
+402	val_402	2008-04-08	11
+403	val_403	2008-04-08	11
+403	val_403	2008-04-08	11
+403	val_403	2008-04-08	11
+404	val_404	2008-04-08	11
+404	val_404	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+406	val_406	2008-04-08	11
+407	val_407	2008-04-08	11
+409	val_409	2008-04-08	11
+409	val_409	2008-04-08	11
+409	val_409	2008-04-08	11
+411	val_411	2008-04-08	11
+413	val_413	2008-04-08	11
+413	val_413	2008-04-08	11
+414	val_414	2008-04-08	11
+414	val_414	2008-04-08	11
+417	val_417	2008-04-08	11
+417	val_417	2008-04-08	11
+417	val_417	2008-04-08	11
+418	val_418	2008-04-08	11
+419	val_419	2008-04-08	11
+421	val_421	2008-04-08	11
+424	val_424	2008-04-08	11
+424	val_424	2008-04-08	11
+427	val_427	2008-04-08	11
+429	val_429	2008-04-08	11
+429	val_429	2008-04-08	11
+430	val_430	2008-04-08	11
+430	val_430	2008-04-08	11
+430	val_430	2008-04-08	11
+431	val_431	2008-04-08	11
+431	val_431	2008-04-08	11
+431	val_431	2008-04-08	11
+432	val_432	2008-04-08	11
+435	val_435	2008-04-08	11
+436	val_436	2008-04-08	11
+437	val_437	2008-04-08	11
+438	val_438	2008-04-08	11
+438	val_438	2008-04-08	11
+438	val_438	2008-04-08	11
+439	val_439	2008-04-08	11
+439	val_439	2008-04-08	11
+443	val_443	2008-04-08	11
+444	val_444	2008-04-08	11
+446	val_446	2008-04-08	11
+448	val_448	2008-04-08	11
+449	val_449	2008-04-08	11
+452	val_452	2008-04-08	11
+453	val_453	2008-04-08	11
+454	val_454	2008-04-08	11
+454	val_454	2008-04-08	11
+454	val_454	2008-04-08	11
+455	val_455	2008-04-08	11
+457	val_457	2008-04-08	11
+458	val_458	2008-04-08	11
+458	val_458	2008-04-08	11
+459	val_459	2008-04-08	11
+459	val_459	2008-04-08	11
+460	val_460	2008-04-08	11
+462	val_462	2008-04-08	11
+462	val_462	2008-04-08	11
+463	val_463	2008-04-08	11
+463	val_463	2008-04-08	11
+466	val_466	2008-04-08	11
+466	val_466	2008-04-08	11
+466	val_466	2008-04-08	11
+467	val_467	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+468	val_468	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+469	val_469	2008-04-08	11
+470	val_470	2008-04-08	11
+472	val_472	2008-04-08	11
+475	val_475	2008-04-08	11
+477	val_477	2008-04-08	11
+478	val_478	2008-04-08	11
+478	val_478	2008-04-08	11
+479	val_479	2008-04-08	11
+480	val_480	2008-04-08	11
+480	val_480	2008-04-08	11
+480	val_480	2008-04-08	11
+481	val_481	2008-04-08	11
+482	val_482	2008-04-08	11
+483	val_483	2008-04-08	11
+484	val_484	2008-04-08	11
+485	val_485	2008-04-08	11
+487	val_487	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+489	val_489	2008-04-08	11
+490	val_490	2008-04-08	11
+491	val_491	2008-04-08	11
+492	val_492	2008-04-08	11
+492	val_492	2008-04-08	11
+493	val_493	2008-04-08	11
+494	val_494	2008-04-08	11
+495	val_495	2008-04-08	11
+496	val_496	2008-04-08	11
+497	val_497	2008-04-08	11
+498	val_498	2008-04-08	11
+498	val_498	2008-04-08	11
+498	val_498	2008-04-08	11
+0	val_0	2008-04-08	12
+0	val_0	2008-04-08	12
+0	val_0	2008-04-08	12
+2	val_2	2008-04-08	12
+4	val_4	2008-04-08	12
+5	val_5	2008-04-08	12
+5	val_5	2008-04-08	12
+5	val_5	2008-04-08	12
+8	val_8	2008-04-08	12
+9	val_9	2008-04-08	12
+10	val_10	2008-04-08	12
+11	val_11	2008-04-08	12
+12	val_12	2008-04-08	12
+12	val_12	2008-04-08	12
+15	val_15	2008-04-08	12
+15	val_15	2008-04-08	12
+17	val_17	2008-04-08	12
+18	val_18	2008-04-08	12
+18	val_18	2008-04-08	12
+19	val_19	2008-04-08	12
+20	val_20	2008-04-08	12
+24	val_24	2008-04-08	12
+24	val_24	2008-04-08	12
+26	val_26	2008-04-08	12
+26	val_26	2008-04-08	12
+27	val_27	2008-04-08	12
+28	val_28	2008-04-08	12
+30	val_30	2008-04-08	12
+33	val_33	2008-04-08	12
+34	val_34	2008-04-08	12
+35	val_35	2008-04-08	12
+35	val_35	2008-04-08	12
+35	val_35	2008-04-08	12
+37	val_37	2008-04-08	12
+37	val_37	2008-04-08	12
+41	val_41	2008-04-08	12
+42	val_42	2008-04-08	12
+42	val_42	2008-04-08	12
+43	val_43	2008-04-08	12
+44	val_44	2008-04-08	12
+47	val_47	2008-04-08	12
+51	val_51	2008-04-08	12
+51	val_51	2008-04-08	12
+53	val_53	2008-04-08	12
+54	val_54	2008-04-08	12
+57	val_57	2008-04-08	12
+58	val_58	2008-04-08	12
+58	val_58	2008-04-08	12
+64	val_64	2008-04-08	12
+65	val_65	2008-04-08	12
+66	val_66	2008-04-08	12
+67	val_67	2008-04-08	12
+67	val_67	2008-04-08	12
+69	val_69	2008-04-08	12
+70	val_70	2008-04-08	12
+70	val_70	2008-04-08	12
+70	val_70	2008-04-08	12
+72	val_72	2008-04-08	12
+72	val_72	2008-04-08	12
+74	val_74	2008-04-08	12
+76	val_76	2008-04-08	12
+76	val_76	2008-04-08	12
+77	val_77	2008-04-08	12
+78	val_78	2008-04-08	12
+80	val_80	2008-04-08	12
+82	val_82	2008-04-08	12
+83	val_83	2008-04-08	12
+83	val_83	2008-04-08	12
+84	val_84	2008-04-08	12
+84	val_84	2008-04-08	12
+85	val_85	2008-04-08	12
+86	val_86	2008-04-08	12
+87	val_87	2008-04-08	12
+90	val_90	2008-04-08	12
+90	val_90	2008-04-08	12
+90	val_90	2008-04-08	12
+92	val_92	2008-04-08	12
+95	val_95	2008-04-08	12
+95	val_95	2008-04-08	12
+96	val_96	2008-04-08	12
+97	val_97	2008-04-08	12
+97	val_97	2008-04-08	12
+98	val_98	2008-04-08	12
+98	val_98	2008-04-08	12
+100	val_100	2008-04-08	12
+100	val_100	2008-04-08	12
+103	val_103	2008-04-08	12
+103	val_103	2008-04-08	12
+104	val_104	2008-04-08	12
+104	val_104	2008-04-08	12
+105	val_105	2008-04-08	12
+111	val_111	2008-04-08	12
+113	val_113	2008-04-08	12
+113	val_113	2008-04-08	12
+114	val_114	2008-04-08	12
+116	val_116	2008-04-08	12
+118	val_118	2008-04-08	12
+118	val_118	2008-04-08	12
+119	val_119	2008-04-08	12
+119	val_119	2008-04-08	12
+119	val_119	2008-04-08	12
+120	val_120	2008-04-08	12
+120	val_120	2008-04-08	12
+125	val_125	2008-04-08	12
+125	val_125	2008-04-08	12
+126	val_126	2008-04-08	12
+128	val_128	2008-04-08	12
+128	val_128	2008-04-08	12
+128	val_128	2008-04-08	12
+129	val_129	2008-04-08	12
+129	val_129	2008-04-08	12
+131	val_131	2008-04-08	12
+133	val_133	2008-04-08	12
+134	val_134	2008-04-08	12
+134	val_134	2008-04-08	12
+136	val_136	2008-04-08	12
+137	val_137	2008-04-08	12
+137	val_137	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+138	val_138	2008-04-08	12
+143	val_143	2008-04-08	12
+145	val_145	2008-04-08	12
+146	val_146	2008-04-08	12
+146	val_146	2008-04-08	12
+149	val_149	2008-04-08	12
+149	val_149	2008-04-08	12
+150	val_150	2008-04-08	12
+152	val_152	2008-04-08	12
+152	val_152	2008-04-08	12
+153	val_153	2008-04-08	12
+155	val_155	2008-04-08	12
+156	val_156	2008-04-08	12
+157	val_157	2008-04-08	12
+158	val_158	2008-04-08	12
+160	val_160	2008-04-08	12
+162	val_162	2008-04-08	12
+163	val_163	2008-04-08	12
+164	val_164	2008-04-08	12
+164	val_164	2008-04-08	12
+165	val_165	2008-04-08	12
+165	val_165	2008-04-08	12
+166	val_166	2008-04-08	12
+167	val_167	2008-04-08	12
+167	val_167	2008-04-08	12
+167	val_167	2008-04-08	12
+168	val_168	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+169	val_169	2008-04-08	12
+170	val_170	2008-04-08	12
+172	val_172	2008-04-08	12
+172	val_172	2008-04-08	12
+174	val_174	2008-04-08	12
+174	val_174	2008-04-08	12
+175	val_175	2008-04-08	12
+175	val_175	2008-04-08	12
+176	val_176	2008-04-08	12
+176	val_176	2008-04-08	12
+177	val_177	2008-04-08	12
+178	val_178	2008-04-08	12
+179	val_179	2008-04-08	12
+179	val_179	2008-04-08	12
+180	val_180	2008-04-08	12
+181	val_181	2008-04-08	12
+183	val_183	2008-04-08	12
+186	val_186	2008-04-08	12
+187	val_187	2008-04-08	12
+187	val_187	2008-04-08	12
+187	val_187	2008-04-08	12
+189	val_189	2008-04-08	12
+190	val_190	2008-04-08	12
+191	val_191	2008-04-08	12
+191	val_191	2008-04-08	12
+192	val_192	2008-04-08	12
+193	val_193	2008-04-08	12
+193	val_193	2008-04-08	12
+193	val_193	2008-04-08	12
+194	val_194	2008-04-08	12
+195	val_195	2008-04-08	12
+195	val_195	2008-04-08	12
+196	val_196	2008-04-08	12
+197	val_197	2008-04-08	12
+197	val_197	2008-04-08	12
+199	val_199	2008-04-08	12
+199	val_199	2008-04-08	12
+199	val_199	2008-04-08	12
+200	val_200	2008-04-08	12
+200	val_200	2008-04-08	12
+201	val_201	2008-04-08	12
+202	val_202	2008-04-08	12
+203	val_203	2008-04-08	12
+203	val_203	2008-04-08	12
+205	val_205	2008-04-08	12
+205	val_205	2008-04-08	12
+207	val_207	2008-04-08	12
+207	val_207	2008-04-08	12
+208	val_208	2008-04-08	12
+208	val_208	2008-04-08	12
+208	val_208	2008-04-08	12
+209	val_209	2008-04-08	12
+209	val_209	2008-04-08	12
+213	val_213	2008-04-08	12
+213	val_213	2008-04-08	12
+214	val_214	2008-04-08	12
+216	val_216	2008-04-08	12
+216	val_216	2008-04-08	12
+217	val_217	2008-04-08	12
+217	val_217	2008-04-08	12
+218	val_218	2008-04-08	12
+219	val_219	2008-04-08	12
+219	val_219	2008-04-08	12
+221	val_221	2008-04-08	12
+221	val_221	2008-04-08	12
+222	val_222	2008-04-08	12
+223	val_223	2008-04-08	12
+223	val_223	2008-04-08	12
+224	val_224	2008-04-08	12
+224	val_224	2008-04-08	12
+226	val_226	2008-04-08	12
+228	val_228	2008-04-08	12
+229	val_229	2008-04-08	12
+229	val_229	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+230	val_230	2008-04-08	12
+233	val_233	2008-04-08	12
+233	val_233	2008-04-08	12
+235	val_235	2008-04-08	12
+237	val_237	2008-04-08	12
+237	val_237	2008-04-08	12
+238	val_238	2008-04-08	12
+238	val_238	2008-04-08	12
+239	val_239	2008-04-08	12
+239	val_239	2008-04-08	12
+241	val_241	2008-04-08	12
+242	val_242	2008-04-08	12
+242	val_242	2008-04-08	12
+244	val_244	2008-04-08	12
+247	val_247	2008-04-08	12
+248	val_248	2008-04-08	12
+249	val_249	2008-04-08	12
+252	val_252	2008-04-08	12
+255	val_255	2008-04-08	12
+255	val_255	2008-04-08	12
+256	val_256	2008-04-08	12
+256	val_256	2008-04-08	12
+257	val_257	2008-04-08	12
+258	val_258	2008-04-08	12
+260	val_260	2008-04-08	12
+262	val_262	2008-04-08	12
+263	val_263	2008-04-08	12
+265	val_265	2008-04-08	12
+265	val_265	2008-04-08	12
+266	val_266	2008-04-08	12
+272	val_272	2008-04-08	12
+272	val_272	2008-04-08	12
+273	val_273	2008-04-08	12
+273	val_273	2008-04-08	12
+273	val_273	2008-04-08	12
+274	val_274	2008-04-08	12
+275	val_275	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+277	val_277	2008-04-08	12
+278	val_278	2008-04-08	12
+278	val_278	2008-04-08	12
+280	val_280	2008-04-08	12
+280	val_280	2008-04-08	12
+281	val_281	2008-04-08	12
+281	val_281	2008-04-08	12
+282	val_282	2008-04-08	12
+282	val_282	2008-04-08	12
+283	val_283	2008-04-08	12
+284	val_284	2008-04-08	12
+285	val_285	2008-04-08	12
+286	val_286	2008-04-08	12
+287	val_287	2008-04-08	12
+288	val_288	2008-04-08	12
+288	val_288	2008-04-08	12
+289	val_289	2008-04-08	12
+291	val_291	2008-04-08	12
+292	val_292	2008-04-08	12
+296	val_296	2008-04-08	12
+298	val_298	2008-04-08	12
+298	val_298	2008-04-08	12
+298	val_298	2008-04-08	12
+302	val_302	2008-04-08	12
+305	val_305	2008-04-08	12
+306	val_306	2008-04-08	12
+307	val_307	2008-04-08	12
+307	val_307	2008-04-08	12
+308	val_308	2008-04-08	12
+309	val_309	2008-04-08	12
+309	val_309	2008-04-08	12
+310	val_310	2008-04-08	12
+311	val_311	2008-04-08	12
+311	val_311	2008-04-08	12
+311	val_311	2008-04-08	12
+315	val_315	2008-04-08	12
+316	val_316	2008-04-08	12
+316	val_316	2008-04-08	12
+316	val_316	2008-04-08	12
+317	val_317	2008-04-08	12
+317	val_317	2008-04-08	12
+318	val_318	2008-04-08	12
+318	val_318	2008-04-08	12
+318	val_318	2008-04-08	12
+321	val_321	2008-04-08	12
+321	val_321	2008-04-08	12
+322	val_322	2008-04-08	12
+322	val_322	2008-04-08	12
+323	val_323	2008-04-08	12
+325	val_325	2008-04-08	12
+325	val_325	2008-04-08	12
+327	val_327	2008-04-08	12
+327	val_327	2008-04-08	12
+327	val_327	2008-04-08	12
+331	val_331	2008-04-08	12
+331	val_331	2008-04-08	12
+332	val_332	2008-04-08	12
+333	val_333	2008-04-08	12
+333	val_333	2008-04-08	12
+335	val_335	2008-04-08	12
+336	val_336	2008-04-08	12
+338	val_338	2008-04-08	12
+339	val_339	2008-04-08	12
+341	val_341	2008-04-08	12
+342	val_342	2008-04-08	12
+342	val_342	2008-04-08	12
+344	val_344	2008-04-08	12
+344	val_344	2008-04-08	12
+345	val_345	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+348	val_348	2008-04-08	12
+351	val_351	2008-04-08	12
+353	val_353	2008-04-08	12
+353	val_353	2008-04-08	12
+356	val_356	2008-04-08	12
+360	val_360	2008-04-08	12
+362	val_362	2008-04-08	12
+364	val_364	2008-04-08	12
+365	val_365	2008-04-08	12
+366	val_366	2008-04-08	12
+367	val_367	2008-04-08	12
+367	val_367	2008-04-08	12
+368	val_368	2008-04-08	12
+369	val_369	2008-04-08	12
+369	val_369	2008-04-08	12
+369	val_369	2008-04-08	12
+373	val_373	2008-04-08	12
+374	val_374	2008-04-08	12
+375	val_375	2008-04-08	12
+377	val_377	2008-04-08	12
+378	val_378	2008-04-08	12
+379	val_379	2008-04-08	12
+382	val_382	2008-04-08	12
+382	val_382	2008-04-08	12
+384	val_384	2008-04-08	12
+384	val_384	2008-04-08	12
+384	val_384	2008-04-08	12
+386	val_386	2008-04-08	12
+389	val_389	2008-04-08	12
+392	val_392	2008-04-08	12
+393	val_393	2008-04-08	12
+394	val_394	2008-04-08	12
+395	val_395	2008-04-08	12
+395	val_395	2008-04-08	12
+396	val_396	2008-04-08	12
+396	val_396	2008-04-08	12
+396	val_396	2008-04-08	12
+397	val_397	2008-04-08	12
+397	val_397	2008-04-08	12
+399	val_399	2008-04-08	12
+399	val_399	2008-04-08	12
+400	val_400	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+401	val_401	2008-04-08	12
+402	val_402	2008-04-08	12
+403	val_403	2008-04-08	12
+403	val_403	2008-04-08	12
+403	val_403	2008-04-08	12
+404	val_404	2008-04-08	12
+404	val_404	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+406	val_406	2008-04-08	12
+407	val_407	2008-04-08	12
+409	val_409	2008-04-08	12
+409	val_409	2008-04-08	12
+409	val_409	2008-04-08	12
+411	val_411	2008-04-08	12
+413	val_413	2008-04-08	12
+413	val_413	2008-04-08	12
+414	val_414	2008-04-08	12
+414	val_414	2008-04-08	12
+417	val_417	2008-04-08	12
+417	val_417	2008-04-08	12
+417	val_417	2008-04-08	12
+418	val_418	2008-04-08	12
+419	val_419	2008-04-08	12
+421	val_421	2008-04-08	12
+424	val_424	2008-04-08	12
+424	val_424	2008-04-08	12
+427	val_427	2008-04-08	12
+429	val_429	2008-04-08	12
+429	val_429	2008-04-08	12
+430	val_430	2008-04-08	12
+430	val_430	2008-04-08	12
+430	val_430	2008-04-08	12
+431	val_431	2008-04-08	12
+431	val_431	2008-04-08	12
+431	val_431	2008-04-08	12
+432	val_432	2008-04-08	12
+435	val_435	2008-04-08	12
+436	val_436	2008-04-08	12
+437	val_437	2008-04-08	12
+438	val_438	2008-04-08	12
+438	val_438	2008-04-08	12
+438	val_438	2008-04-08	12
+439	val_439	2008-04-08	12
+439	val_439	2008-04-08	12
+443	val_443	2008-04-08	12
+444	val_444	2008-04-08	12
+446	val_446	2008-04-08	12
+448	val_448	2008-04-08	12
+449	val_449	2008-04-08	12
+452	val_452	2008-04-08	12
+453	val_453	2008-04-08	12
+454	val_454	2008-04-08	12
+454	val_454	2008-04-08	12
+454	val_454	2008-04-08	12
+455	val_455	2008-04-08	12
+457	val_457	2008-04-08	12
+458	val_458	2008-04-08	12
+458	val_458	2008-04-08	12
+459	val_459	2008-04-08	12
+459	val_459	2008-04-08	12
+460	val_460	2008-04-08	12
+462	val_462	2008-04-08	12
+462	val_462	2008-04-08	12
+463	val_463	2008-04-08	12
+463	val_463	2008-04-08	12
+466	val_466	2008-04-08	12
+466	val_466	2008-04-08	12
+466	val_466	2008-04-08	12
+467	val_467	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+468	val_468	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+469	val_469	2008-04-08	12
+470	val_470	2008-04-08	12
+472	val_472	2008-04-08	12
+475	val_475	2008-04-08	12
+477	val_477	2008-04-08	12
+478	val_478	2008-04-08	12
+478	val_478	2008-04-08	12
+479	val_479	2008-04-08	12
+480	val_480	2008-04-08	12
+480	val_480	2008-04-08	12
+480	val_480	2008-04-08	12
+481	val_481	2008-04-08	12
+482	val_482	2008-04-08	12
+483	val_483	2008-04-08	12
+484	val_484	2008-04-08	12
+485	val_485	2008-04-08	12
+487	val_487	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+489	val_489	2008-04-08	12
+490	val_490	2008-04-08	12
+491	val_491	2008-04-08	12
+492	val_492	2008-04-08	12
+492	val_492	2008-04-08	12
+493	val_493	2008-04-08	12
+494	val_494	2008-04-08	12
+495	val_495	2008-04-08	12
+496	val_496	2008-04-08	12
+497	val_497	2008-04-08	12
+498	val_498	2008-04-08	12
+498	val_498	2008-04-08	12
+498	val_498	2008-04-08	12
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb b/sql/hive/src/test/resources/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb
deleted file mode 100644
index d7a8f25b41301ec8d6a1bd3d230e31a65586444e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb
+++ /dev/null
@@ -1,1000 +0,0 @@
-238	val_238	2008-04-08	11
-86	val_86	2008-04-08	11
-311	val_311	2008-04-08	11
-27	val_27	2008-04-08	11
-165	val_165	2008-04-08	11
-409	val_409	2008-04-08	11
-255	val_255	2008-04-08	11
-278	val_278	2008-04-08	11
-98	val_98	2008-04-08	11
-484	val_484	2008-04-08	11
-265	val_265	2008-04-08	11
-193	val_193	2008-04-08	11
-401	val_401	2008-04-08	11
-150	val_150	2008-04-08	11
-273	val_273	2008-04-08	11
-224	val_224	2008-04-08	11
-369	val_369	2008-04-08	11
-66	val_66	2008-04-08	11
-128	val_128	2008-04-08	11
-213	val_213	2008-04-08	11
-146	val_146	2008-04-08	11
-406	val_406	2008-04-08	11
-429	val_429	2008-04-08	11
-374	val_374	2008-04-08	11
-152	val_152	2008-04-08	11
-469	val_469	2008-04-08	11
-145	val_145	2008-04-08	11
-495	val_495	2008-04-08	11
-37	val_37	2008-04-08	11
-327	val_327	2008-04-08	11
-281	val_281	2008-04-08	11
-277	val_277	2008-04-08	11
-209	val_209	2008-04-08	11
-15	val_15	2008-04-08	11
-82	val_82	2008-04-08	11
-403	val_403	2008-04-08	11
-166	val_166	2008-04-08	11
-417	val_417	2008-04-08	11
-430	val_430	2008-04-08	11
-252	val_252	2008-04-08	11
-292	val_292	2008-04-08	11
-219	val_219	2008-04-08	11
-287	val_287	2008-04-08	11
-153	val_153	2008-04-08	11
-193	val_193	2008-04-08	11
-338	val_338	2008-04-08	11
-446	val_446	2008-04-08	11
-459	val_459	2008-04-08	11
-394	val_394	2008-04-08	11
-237	val_237	2008-04-08	11
-482	val_482	2008-04-08	11
-174	val_174	2008-04-08	11
-413	val_413	2008-04-08	11
-494	val_494	2008-04-08	11
-207	val_207	2008-04-08	11
-199	val_199	2008-04-08	11
-466	val_466	2008-04-08	11
-208	val_208	2008-04-08	11
-174	val_174	2008-04-08	11
-399	val_399	2008-04-08	11
-396	val_396	2008-04-08	11
-247	val_247	2008-04-08	11
-417	val_417	2008-04-08	11
-489	val_489	2008-04-08	11
-162	val_162	2008-04-08	11
-377	val_377	2008-04-08	11
-397	val_397	2008-04-08	11
-309	val_309	2008-04-08	11
-365	val_365	2008-04-08	11
-266	val_266	2008-04-08	11
-439	val_439	2008-04-08	11
-342	val_342	2008-04-08	11
-367	val_367	2008-04-08	11
-325	val_325	2008-04-08	11
-167	val_167	2008-04-08	11
-195	val_195	2008-04-08	11
-475	val_475	2008-04-08	11
-17	val_17	2008-04-08	11
-113	val_113	2008-04-08	11
-155	val_155	2008-04-08	11
-203	val_203	2008-04-08	11
-339	val_339	2008-04-08	11
-0	val_0	2008-04-08	11
-455	val_455	2008-04-08	11
-128	val_128	2008-04-08	11
-311	val_311	2008-04-08	11
-316	val_316	2008-04-08	11
-57	val_57	2008-04-08	11
-302	val_302	2008-04-08	11
-205	val_205	2008-04-08	11
-149	val_149	2008-04-08	11
-438	val_438	2008-04-08	11
-345	val_345	2008-04-08	11
-129	val_129	2008-04-08	11
-170	val_170	2008-04-08	11
-20	val_20	2008-04-08	11
-489	val_489	2008-04-08	11
-157	val_157	2008-04-08	11
-378	val_378	2008-04-08	11
-221	val_221	2008-04-08	11
-92	val_92	2008-04-08	11
-111	val_111	2008-04-08	11
-47	val_47	2008-04-08	11
-72	val_72	2008-04-08	11
-4	val_4	2008-04-08	11
-280	val_280	2008-04-08	11
-35	val_35	2008-04-08	11
-427	val_427	2008-04-08	11
-277	val_277	2008-04-08	11
-208	val_208	2008-04-08	11
-356	val_356	2008-04-08	11
-399	val_399	2008-04-08	11
-169	val_169	2008-04-08	11
-382	val_382	2008-04-08	11
-498	val_498	2008-04-08	11
-125	val_125	2008-04-08	11
-386	val_386	2008-04-08	11
-437	val_437	2008-04-08	11
-469	val_469	2008-04-08	11
-192	val_192	2008-04-08	11
-286	val_286	2008-04-08	11
-187	val_187	2008-04-08	11
-176	val_176	2008-04-08	11
-54	val_54	2008-04-08	11
-459	val_459	2008-04-08	11
-51	val_51	2008-04-08	11
-138	val_138	2008-04-08	11
-103	val_103	2008-04-08	11
-239	val_239	2008-04-08	11
-213	val_213	2008-04-08	11
-216	val_216	2008-04-08	11
-430	val_430	2008-04-08	11
-278	val_278	2008-04-08	11
-176	val_176	2008-04-08	11
-289	val_289	2008-04-08	11
-221	val_221	2008-04-08	11
-65	val_65	2008-04-08	11
-318	val_318	2008-04-08	11
-332	val_332	2008-04-08	11
-311	val_311	2008-04-08	11
-275	val_275	2008-04-08	11
-137	val_137	2008-04-08	11
-241	val_241	2008-04-08	11
-83	val_83	2008-04-08	11
-333	val_333	2008-04-08	11
-180	val_180	2008-04-08	11
-284	val_284	2008-04-08	11
-12	val_12	2008-04-08	11
-230	val_230	2008-04-08	11
-181	val_181	2008-04-08	11
-67	val_67	2008-04-08	11
-260	val_260	2008-04-08	11
-404	val_404	2008-04-08	11
-384	val_384	2008-04-08	11
-489	val_489	2008-04-08	11
-353	val_353	2008-04-08	11
-373	val_373	2008-04-08	11
-272	val_272	2008-04-08	11
-138	val_138	2008-04-08	11
-217	val_217	2008-04-08	11
-84	val_84	2008-04-08	11
-348	val_348	2008-04-08	11
-466	val_466	2008-04-08	11
-58	val_58	2008-04-08	11
-8	val_8	2008-04-08	11
-411	val_411	2008-04-08	11
-230	val_230	2008-04-08	11
-208	val_208	2008-04-08	11
-348	val_348	2008-04-08	11
-24	val_24	2008-04-08	11
-463	val_463	2008-04-08	11
-431	val_431	2008-04-08	11
-179	val_179	2008-04-08	11
-172	val_172	2008-04-08	11
-42	val_42	2008-04-08	11
-129	val_129	2008-04-08	11
-158	val_158	2008-04-08	11
-119	val_119	2008-04-08	11
-496	val_496	2008-04-08	11
-0	val_0	2008-04-08	11
-322	val_322	2008-04-08	11
-197	val_197	2008-04-08	11
-468	val_468	2008-04-08	11
-393	val_393	2008-04-08	11
-454	val_454	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-199	val_199	2008-04-08	11
-191	val_191	2008-04-08	11
-418	val_418	2008-04-08	11
-96	val_96	2008-04-08	11
-26	val_26	2008-04-08	11
-165	val_165	2008-04-08	11
-327	val_327	2008-04-08	11
-230	val_230	2008-04-08	11
-205	val_205	2008-04-08	11
-120	val_120	2008-04-08	11
-131	val_131	2008-04-08	11
-51	val_51	2008-04-08	11
-404	val_404	2008-04-08	11
-43	val_43	2008-04-08	11
-436	val_436	2008-04-08	11
-156	val_156	2008-04-08	11
-469	val_469	2008-04-08	11
-468	val_468	2008-04-08	11
-308	val_308	2008-04-08	11
-95	val_95	2008-04-08	11
-196	val_196	2008-04-08	11
-288	val_288	2008-04-08	11
-481	val_481	2008-04-08	11
-457	val_457	2008-04-08	11
-98	val_98	2008-04-08	11
-282	val_282	2008-04-08	11
-197	val_197	2008-04-08	11
-187	val_187	2008-04-08	11
-318	val_318	2008-04-08	11
-318	val_318	2008-04-08	11
-409	val_409	2008-04-08	11
-470	val_470	2008-04-08	11
-137	val_137	2008-04-08	11
-369	val_369	2008-04-08	11
-316	val_316	2008-04-08	11
-169	val_169	2008-04-08	11
-413	val_413	2008-04-08	11
-85	val_85	2008-04-08	11
-77	val_77	2008-04-08	11
-0	val_0	2008-04-08	11
-490	val_490	2008-04-08	11
-87	val_87	2008-04-08	11
-364	val_364	2008-04-08	11
-179	val_179	2008-04-08	11
-118	val_118	2008-04-08	11
-134	val_134	2008-04-08	11
-395	val_395	2008-04-08	11
-282	val_282	2008-04-08	11
-138	val_138	2008-04-08	11
-238	val_238	2008-04-08	11
-419	val_419	2008-04-08	11
-15	val_15	2008-04-08	11
-118	val_118	2008-04-08	11
-72	val_72	2008-04-08	11
-90	val_90	2008-04-08	11
-307	val_307	2008-04-08	11
-19	val_19	2008-04-08	11
-435	val_435	2008-04-08	11
-10	val_10	2008-04-08	11
-277	val_277	2008-04-08	11
-273	val_273	2008-04-08	11
-306	val_306	2008-04-08	11
-224	val_224	2008-04-08	11
-309	val_309	2008-04-08	11
-389	val_389	2008-04-08	11
-327	val_327	2008-04-08	11
-242	val_242	2008-04-08	11
-369	val_369	2008-04-08	11
-392	val_392	2008-04-08	11
-272	val_272	2008-04-08	11
-331	val_331	2008-04-08	11
-401	val_401	2008-04-08	11
-242	val_242	2008-04-08	11
-452	val_452	2008-04-08	11
-177	val_177	2008-04-08	11
-226	val_226	2008-04-08	11
-5	val_5	2008-04-08	11
-497	val_497	2008-04-08	11
-402	val_402	2008-04-08	11
-396	val_396	2008-04-08	11
-317	val_317	2008-04-08	11
-395	val_395	2008-04-08	11
-58	val_58	2008-04-08	11
-35	val_35	2008-04-08	11
-336	val_336	2008-04-08	11
-95	val_95	2008-04-08	11
-11	val_11	2008-04-08	11
-168	val_168	2008-04-08	11
-34	val_34	2008-04-08	11
-229	val_229	2008-04-08	11
-233	val_233	2008-04-08	11
-143	val_143	2008-04-08	11
-472	val_472	2008-04-08	11
-322	val_322	2008-04-08	11
-498	val_498	2008-04-08	11
-160	val_160	2008-04-08	11
-195	val_195	2008-04-08	11
-42	val_42	2008-04-08	11
-321	val_321	2008-04-08	11
-430	val_430	2008-04-08	11
-119	val_119	2008-04-08	11
-489	val_489	2008-04-08	11
-458	val_458	2008-04-08	11
-78	val_78	2008-04-08	11
-76	val_76	2008-04-08	11
-41	val_41	2008-04-08	11
-223	val_223	2008-04-08	11
-492	val_492	2008-04-08	11
-149	val_149	2008-04-08	11
-449	val_449	2008-04-08	11
-218	val_218	2008-04-08	11
-228	val_228	2008-04-08	11
-138	val_138	2008-04-08	11
-453	val_453	2008-04-08	11
-30	val_30	2008-04-08	11
-209	val_209	2008-04-08	11
-64	val_64	2008-04-08	11
-468	val_468	2008-04-08	11
-76	val_76	2008-04-08	11
-74	val_74	2008-04-08	11
-342	val_342	2008-04-08	11
-69	val_69	2008-04-08	11
-230	val_230	2008-04-08	11
-33	val_33	2008-04-08	11
-368	val_368	2008-04-08	11
-103	val_103	2008-04-08	11
-296	val_296	2008-04-08	11
-113	val_113	2008-04-08	11
-216	val_216	2008-04-08	11
-367	val_367	2008-04-08	11
-344	val_344	2008-04-08	11
-167	val_167	2008-04-08	11
-274	val_274	2008-04-08	11
-219	val_219	2008-04-08	11
-239	val_239	2008-04-08	11
-485	val_485	2008-04-08	11
-116	val_116	2008-04-08	11
-223	val_223	2008-04-08	11
-256	val_256	2008-04-08	11
-263	val_263	2008-04-08	11
-70	val_70	2008-04-08	11
-487	val_487	2008-04-08	11
-480	val_480	2008-04-08	11
-401	val_401	2008-04-08	11
-288	val_288	2008-04-08	11
-191	val_191	2008-04-08	11
-5	val_5	2008-04-08	11
-244	val_244	2008-04-08	11
-438	val_438	2008-04-08	11
-128	val_128	2008-04-08	11
-467	val_467	2008-04-08	11
-432	val_432	2008-04-08	11
-202	val_202	2008-04-08	11
-316	val_316	2008-04-08	11
-229	val_229	2008-04-08	11
-469	val_469	2008-04-08	11
-463	val_463	2008-04-08	11
-280	val_280	2008-04-08	11
-2	val_2	2008-04-08	11
-35	val_35	2008-04-08	11
-283	val_283	2008-04-08	11
-331	val_331	2008-04-08	11
-235	val_235	2008-04-08	11
-80	val_80	2008-04-08	11
-44	val_44	2008-04-08	11
-193	val_193	2008-04-08	11
-321	val_321	2008-04-08	11
-335	val_335	2008-04-08	11
-104	val_104	2008-04-08	11
-466	val_466	2008-04-08	11
-366	val_366	2008-04-08	11
-175	val_175	2008-04-08	11
-403	val_403	2008-04-08	11
-483	val_483	2008-04-08	11
-53	val_53	2008-04-08	11
-105	val_105	2008-04-08	11
-257	val_257	2008-04-08	11
-406	val_406	2008-04-08	11
-409	val_409	2008-04-08	11
-190	val_190	2008-04-08	11
-406	val_406	2008-04-08	11
-401	val_401	2008-04-08	11
-114	val_114	2008-04-08	11
-258	val_258	2008-04-08	11
-90	val_90	2008-04-08	11
-203	val_203	2008-04-08	11
-262	val_262	2008-04-08	11
-348	val_348	2008-04-08	11
-424	val_424	2008-04-08	11
-12	val_12	2008-04-08	11
-396	val_396	2008-04-08	11
-201	val_201	2008-04-08	11
-217	val_217	2008-04-08	11
-164	val_164	2008-04-08	11
-431	val_431	2008-04-08	11
-454	val_454	2008-04-08	11
-478	val_478	2008-04-08	11
-298	val_298	2008-04-08	11
-125	val_125	2008-04-08	11
-431	val_431	2008-04-08	11
-164	val_164	2008-04-08	11
-424	val_424	2008-04-08	11
-187	val_187	2008-04-08	11
-382	val_382	2008-04-08	11
-5	val_5	2008-04-08	11
-70	val_70	2008-04-08	11
-397	val_397	2008-04-08	11
-480	val_480	2008-04-08	11
-291	val_291	2008-04-08	11
-24	val_24	2008-04-08	11
-351	val_351	2008-04-08	11
-255	val_255	2008-04-08	11
-104	val_104	2008-04-08	11
-70	val_70	2008-04-08	11
-163	val_163	2008-04-08	11
-438	val_438	2008-04-08	11
-119	val_119	2008-04-08	11
-414	val_414	2008-04-08	11
-200	val_200	2008-04-08	11
-491	val_491	2008-04-08	11
-237	val_237	2008-04-08	11
-439	val_439	2008-04-08	11
-360	val_360	2008-04-08	11
-248	val_248	2008-04-08	11
-479	val_479	2008-04-08	11
-305	val_305	2008-04-08	11
-417	val_417	2008-04-08	11
-199	val_199	2008-04-08	11
-444	val_444	2008-04-08	11
-120	val_120	2008-04-08	11
-429	val_429	2008-04-08	11
-169	val_169	2008-04-08	11
-443	val_443	2008-04-08	11
-323	val_323	2008-04-08	11
-325	val_325	2008-04-08	11
-277	val_277	2008-04-08	11
-230	val_230	2008-04-08	11
-478	val_478	2008-04-08	11
-178	val_178	2008-04-08	11
-468	val_468	2008-04-08	11
-310	val_310	2008-04-08	11
-317	val_317	2008-04-08	11
-333	val_333	2008-04-08	11
-493	val_493	2008-04-08	11
-460	val_460	2008-04-08	11
-207	val_207	2008-04-08	11
-249	val_249	2008-04-08	11
-265	val_265	2008-04-08	11
-480	val_480	2008-04-08	11
-83	val_83	2008-04-08	11
-136	val_136	2008-04-08	11
-353	val_353	2008-04-08	11
-172	val_172	2008-04-08	11
-214	val_214	2008-04-08	11
-462	val_462	2008-04-08	11
-233	val_233	2008-04-08	11
-406	val_406	2008-04-08	11
-133	val_133	2008-04-08	11
-175	val_175	2008-04-08	11
-189	val_189	2008-04-08	11
-454	val_454	2008-04-08	11
-375	val_375	2008-04-08	11
-401	val_401	2008-04-08	11
-421	val_421	2008-04-08	11
-407	val_407	2008-04-08	11
-384	val_384	2008-04-08	11
-256	val_256	2008-04-08	11
-26	val_26	2008-04-08	11
-134	val_134	2008-04-08	11
-67	val_67	2008-04-08	11
-384	val_384	2008-04-08	11
-379	val_379	2008-04-08	11
-18	val_18	2008-04-08	11
-462	val_462	2008-04-08	11
-492	val_492	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-9	val_9	2008-04-08	11
-341	val_341	2008-04-08	11
-498	val_498	2008-04-08	11
-146	val_146	2008-04-08	11
-458	val_458	2008-04-08	11
-362	val_362	2008-04-08	11
-186	val_186	2008-04-08	11
-285	val_285	2008-04-08	11
-348	val_348	2008-04-08	11
-167	val_167	2008-04-08	11
-18	val_18	2008-04-08	11
-273	val_273	2008-04-08	11
-183	val_183	2008-04-08	11
-281	val_281	2008-04-08	11
-344	val_344	2008-04-08	11
-97	val_97	2008-04-08	11
-469	val_469	2008-04-08	11
-315	val_315	2008-04-08	11
-84	val_84	2008-04-08	11
-28	val_28	2008-04-08	11
-37	val_37	2008-04-08	11
-448	val_448	2008-04-08	11
-152	val_152	2008-04-08	11
-348	val_348	2008-04-08	11
-307	val_307	2008-04-08	11
-194	val_194	2008-04-08	11
-414	val_414	2008-04-08	11
-477	val_477	2008-04-08	11
-222	val_222	2008-04-08	11
-126	val_126	2008-04-08	11
-90	val_90	2008-04-08	11
-169	val_169	2008-04-08	11
-403	val_403	2008-04-08	11
-400	val_400	2008-04-08	11
-200	val_200	2008-04-08	11
-97	val_97	2008-04-08	11
-238	val_238	2008-04-08	12
-86	val_86	2008-04-08	12
-311	val_311	2008-04-08	12
-27	val_27	2008-04-08	12
-165	val_165	2008-04-08	12
-409	val_409	2008-04-08	12
-255	val_255	2008-04-08	12
-278	val_278	2008-04-08	12
-98	val_98	2008-04-08	12
-484	val_484	2008-04-08	12
-265	val_265	2008-04-08	12
-193	val_193	2008-04-08	12
-401	val_401	2008-04-08	12
-150	val_150	2008-04-08	12
-273	val_273	2008-04-08	12
-224	val_224	2008-04-08	12
-369	val_369	2008-04-08	12
-66	val_66	2008-04-08	12
-128	val_128	2008-04-08	12
-213	val_213	2008-04-08	12
-146	val_146	2008-04-08	12
-406	val_406	2008-04-08	12
-429	val_429	2008-04-08	12
-374	val_374	2008-04-08	12
-152	val_152	2008-04-08	12
-469	val_469	2008-04-08	12
-145	val_145	2008-04-08	12
-495	val_495	2008-04-08	12
-37	val_37	2008-04-08	12
-327	val_327	2008-04-08	12
-281	val_281	2008-04-08	12
-277	val_277	2008-04-08	12
-209	val_209	2008-04-08	12
-15	val_15	2008-04-08	12
-82	val_82	2008-04-08	12
-403	val_403	2008-04-08	12
-166	val_166	2008-04-08	12
-417	val_417	2008-04-08	12
-430	val_430	2008-04-08	12
-252	val_252	2008-04-08	12
-292	val_292	2008-04-08	12
-219	val_219	2008-04-08	12
-287	val_287	2008-04-08	12
-153	val_153	2008-04-08	12
-193	val_193	2008-04-08	12
-338	val_338	2008-04-08	12
-446	val_446	2008-04-08	12
-459	val_459	2008-04-08	12
-394	val_394	2008-04-08	12
-237	val_237	2008-04-08	12
-482	val_482	2008-04-08	12
-174	val_174	2008-04-08	12
-413	val_413	2008-04-08	12
-494	val_494	2008-04-08	12
-207	val_207	2008-04-08	12
-199	val_199	2008-04-08	12
-466	val_466	2008-04-08	12
-208	val_208	2008-04-08	12
-174	val_174	2008-04-08	12
-399	val_399	2008-04-08	12
-396	val_396	2008-04-08	12
-247	val_247	2008-04-08	12
-417	val_417	2008-04-08	12
-489	val_489	2008-04-08	12
-162	val_162	2008-04-08	12
-377	val_377	2008-04-08	12
-397	val_397	2008-04-08	12
-309	val_309	2008-04-08	12
-365	val_365	2008-04-08	12
-266	val_266	2008-04-08	12
-439	val_439	2008-04-08	12
-342	val_342	2008-04-08	12
-367	val_367	2008-04-08	12
-325	val_325	2008-04-08	12
-167	val_167	2008-04-08	12
-195	val_195	2008-04-08	12
-475	val_475	2008-04-08	12
-17	val_17	2008-04-08	12
-113	val_113	2008-04-08	12
-155	val_155	2008-04-08	12
-203	val_203	2008-04-08	12
-339	val_339	2008-04-08	12
-0	val_0	2008-04-08	12
-455	val_455	2008-04-08	12
-128	val_128	2008-04-08	12
-311	val_311	2008-04-08	12
-316	val_316	2008-04-08	12
-57	val_57	2008-04-08	12
-302	val_302	2008-04-08	12
-205	val_205	2008-04-08	12
-149	val_149	2008-04-08	12
-438	val_438	2008-04-08	12
-345	val_345	2008-04-08	12
-129	val_129	2008-04-08	12
-170	val_170	2008-04-08	12
-20	val_20	2008-04-08	12
-489	val_489	2008-04-08	12
-157	val_157	2008-04-08	12
-378	val_378	2008-04-08	12
-221	val_221	2008-04-08	12
-92	val_92	2008-04-08	12
-111	val_111	2008-04-08	12
-47	val_47	2008-04-08	12
-72	val_72	2008-04-08	12
-4	val_4	2008-04-08	12
-280	val_280	2008-04-08	12
-35	val_35	2008-04-08	12
-427	val_427	2008-04-08	12
-277	val_277	2008-04-08	12
-208	val_208	2008-04-08	12
-356	val_356	2008-04-08	12
-399	val_399	2008-04-08	12
-169	val_169	2008-04-08	12
-382	val_382	2008-04-08	12
-498	val_498	2008-04-08	12
-125	val_125	2008-04-08	12
-386	val_386	2008-04-08	12
-437	val_437	2008-04-08	12
-469	val_469	2008-04-08	12
-192	val_192	2008-04-08	12
-286	val_286	2008-04-08	12
-187	val_187	2008-04-08	12
-176	val_176	2008-04-08	12
-54	val_54	2008-04-08	12
-459	val_459	2008-04-08	12
-51	val_51	2008-04-08	12
-138	val_138	2008-04-08	12
-103	val_103	2008-04-08	12
-239	val_239	2008-04-08	12
-213	val_213	2008-04-08	12
-216	val_216	2008-04-08	12
-430	val_430	2008-04-08	12
-278	val_278	2008-04-08	12
-176	val_176	2008-04-08	12
-289	val_289	2008-04-08	12
-221	val_221	2008-04-08	12
-65	val_65	2008-04-08	12
-318	val_318	2008-04-08	12
-332	val_332	2008-04-08	12
-311	val_311	2008-04-08	12
-275	val_275	2008-04-08	12
-137	val_137	2008-04-08	12
-241	val_241	2008-04-08	12
-83	val_83	2008-04-08	12
-333	val_333	2008-04-08	12
-180	val_180	2008-04-08	12
-284	val_284	2008-04-08	12
-12	val_12	2008-04-08	12
-230	val_230	2008-04-08	12
-181	val_181	2008-04-08	12
-67	val_67	2008-04-08	12
-260	val_260	2008-04-08	12
-404	val_404	2008-04-08	12
-384	val_384	2008-04-08	12
-489	val_489	2008-04-08	12
-353	val_353	2008-04-08	12
-373	val_373	2008-04-08	12
-272	val_272	2008-04-08	12
-138	val_138	2008-04-08	12
-217	val_217	2008-04-08	12
-84	val_84	2008-04-08	12
-348	val_348	2008-04-08	12
-466	val_466	2008-04-08	12
-58	val_58	2008-04-08	12
-8	val_8	2008-04-08	12
-411	val_411	2008-04-08	12
-230	val_230	2008-04-08	12
-208	val_208	2008-04-08	12
-348	val_348	2008-04-08	12
-24	val_24	2008-04-08	12
-463	val_463	2008-04-08	12
-431	val_431	2008-04-08	12
-179	val_179	2008-04-08	12
-172	val_172	2008-04-08	12
-42	val_42	2008-04-08	12
-129	val_129	2008-04-08	12
-158	val_158	2008-04-08	12
-119	val_119	2008-04-08	12
-496	val_496	2008-04-08	12
-0	val_0	2008-04-08	12
-322	val_322	2008-04-08	12
-197	val_197	2008-04-08	12
-468	val_468	2008-04-08	12
-393	val_393	2008-04-08	12
-454	val_454	2008-04-08	12
-100	val_100	2008-04-08	12
-298	val_298	2008-04-08	12
-199	val_199	2008-04-08	12
-191	val_191	2008-04-08	12
-418	val_418	2008-04-08	12
-96	val_96	2008-04-08	12
-26	val_26	2008-04-08	12
-165	val_165	2008-04-08	12
-327	val_327	2008-04-08	12
-230	val_230	2008-04-08	12
-205	val_205	2008-04-08	12
-120	val_120	2008-04-08	12
-131	val_131	2008-04-08	12
-51	val_51	2008-04-08	12
-404	val_404	2008-04-08	12
-43	val_43	2008-04-08	12
-436	val_436	2008-04-08	12
-156	val_156	2008-04-08	12
-469	val_469	2008-04-08	12
-468	val_468	2008-04-08	12
-308	val_308	2008-04-08	12
-95	val_95	2008-04-08	12
-196	val_196	2008-04-08	12
-288	val_288	2008-04-08	12
-481	val_481	2008-04-08	12
-457	val_457	2008-04-08	12
-98	val_98	2008-04-08	12
-282	val_282	2008-04-08	12
-197	val_197	2008-04-08	12
-187	val_187	2008-04-08	12
-318	val_318	2008-04-08	12
-318	val_318	2008-04-08	12
-409	val_409	2008-04-08	12
-470	val_470	2008-04-08	12
-137	val_137	2008-04-08	12
-369	val_369	2008-04-08	12
-316	val_316	2008-04-08	12
-169	val_169	2008-04-08	12
-413	val_413	2008-04-08	12
-85	val_85	2008-04-08	12
-77	val_77	2008-04-08	12
-0	val_0	2008-04-08	12
-490	val_490	2008-04-08	12
-87	val_87	2008-04-08	12
-364	val_364	2008-04-08	12
-179	val_179	2008-04-08	12
-118	val_118	2008-04-08	12
-134	val_134	2008-04-08	12
-395	val_395	2008-04-08	12
-282	val_282	2008-04-08	12
-138	val_138	2008-04-08	12
-238	val_238	2008-04-08	12
-419	val_419	2008-04-08	12
-15	val_15	2008-04-08	12
-118	val_118	2008-04-08	12
-72	val_72	2008-04-08	12
-90	val_90	2008-04-08	12
-307	val_307	2008-04-08	12
-19	val_19	2008-04-08	12
-435	val_435	2008-04-08	12
-10	val_10	2008-04-08	12
-277	val_277	2008-04-08	12
-273	val_273	2008-04-08	12
-306	val_306	2008-04-08	12
-224	val_224	2008-04-08	12
-309	val_309	2008-04-08	12
-389	val_389	2008-04-08	12
-327	val_327	2008-04-08	12
-242	val_242	2008-04-08	12
-369	val_369	2008-04-08	12
-392	val_392	2008-04-08	12
-272	val_272	2008-04-08	12
-331	val_331	2008-04-08	12
-401	val_401	2008-04-08	12
-242	val_242	2008-04-08	12
-452	val_452	2008-04-08	12
-177	val_177	2008-04-08	12
-226	val_226	2008-04-08	12
-5	val_5	2008-04-08	12
-497	val_497	2008-04-08	12
-402	val_402	2008-04-08	12
-396	val_396	2008-04-08	12
-317	val_317	2008-04-08	12
-395	val_395	2008-04-08	12
-58	val_58	2008-04-08	12
-35	val_35	2008-04-08	12
-336	val_336	2008-04-08	12
-95	val_95	2008-04-08	12
-11	val_11	2008-04-08	12
-168	val_168	2008-04-08	12
-34	val_34	2008-04-08	12
-229	val_229	2008-04-08	12
-233	val_233	2008-04-08	12
-143	val_143	2008-04-08	12
-472	val_472	2008-04-08	12
-322	val_322	2008-04-08	12
-498	val_498	2008-04-08	12
-160	val_160	2008-04-08	12
-195	val_195	2008-04-08	12
-42	val_42	2008-04-08	12
-321	val_321	2008-04-08	12
-430	val_430	2008-04-08	12
-119	val_119	2008-04-08	12
-489	val_489	2008-04-08	12
-458	val_458	2008-04-08	12
-78	val_78	2008-04-08	12
-76	val_76	2008-04-08	12
-41	val_41	2008-04-08	12
-223	val_223	2008-04-08	12
-492	val_492	2008-04-08	12
-149	val_149	2008-04-08	12
-449	val_449	2008-04-08	12
-218	val_218	2008-04-08	12
-228	val_228	2008-04-08	12
-138	val_138	2008-04-08	12
-453	val_453	2008-04-08	12
-30	val_30	2008-04-08	12
-209	val_209	2008-04-08	12
-64	val_64	2008-04-08	12
-468	val_468	2008-04-08	12
-76	val_76	2008-04-08	12
-74	val_74	2008-04-08	12
-342	val_342	2008-04-08	12
-69	val_69	2008-04-08	12
-230	val_230	2008-04-08	12
-33	val_33	2008-04-08	12
-368	val_368	2008-04-08	12
-103	val_103	2008-04-08	12
-296	val_296	2008-04-08	12
-113	val_113	2008-04-08	12
-216	val_216	2008-04-08	12
-367	val_367	2008-04-08	12
-344	val_344	2008-04-08	12
-167	val_167	2008-04-08	12
-274	val_274	2008-04-08	12
-219	val_219	2008-04-08	12
-239	val_239	2008-04-08	12
-485	val_485	2008-04-08	12
-116	val_116	2008-04-08	12
-223	val_223	2008-04-08	12
-256	val_256	2008-04-08	12
-263	val_263	2008-04-08	12
-70	val_70	2008-04-08	12
-487	val_487	2008-04-08	12
-480	val_480	2008-04-08	12
-401	val_401	2008-04-08	12
-288	val_288	2008-04-08	12
-191	val_191	2008-04-08	12
-5	val_5	2008-04-08	12
-244	val_244	2008-04-08	12
-438	val_438	2008-04-08	12
-128	val_128	2008-04-08	12
-467	val_467	2008-04-08	12
-432	val_432	2008-04-08	12
-202	val_202	2008-04-08	12
-316	val_316	2008-04-08	12
-229	val_229	2008-04-08	12
-469	val_469	2008-04-08	12
-463	val_463	2008-04-08	12
-280	val_280	2008-04-08	12
-2	val_2	2008-04-08	12
-35	val_35	2008-04-08	12
-283	val_283	2008-04-08	12
-331	val_331	2008-04-08	12
-235	val_235	2008-04-08	12
-80	val_80	2008-04-08	12
-44	val_44	2008-04-08	12
-193	val_193	2008-04-08	12
-321	val_321	2008-04-08	12
-335	val_335	2008-04-08	12
-104	val_104	2008-04-08	12
-466	val_466	2008-04-08	12
-366	val_366	2008-04-08	12
-175	val_175	2008-04-08	12
-403	val_403	2008-04-08	12
-483	val_483	2008-04-08	12
-53	val_53	2008-04-08	12
-105	val_105	2008-04-08	12
-257	val_257	2008-04-08	12
-406	val_406	2008-04-08	12
-409	val_409	2008-04-08	12
-190	val_190	2008-04-08	12
-406	val_406	2008-04-08	12
-401	val_401	2008-04-08	12
-114	val_114	2008-04-08	12
-258	val_258	2008-04-08	12
-90	val_90	2008-04-08	12
-203	val_203	2008-04-08	12
-262	val_262	2008-04-08	12
-348	val_348	2008-04-08	12
-424	val_424	2008-04-08	12
-12	val_12	2008-04-08	12
-396	val_396	2008-04-08	12
-201	val_201	2008-04-08	12
-217	val_217	2008-04-08	12
-164	val_164	2008-04-08	12
-431	val_431	2008-04-08	12
-454	val_454	2008-04-08	12
-478	val_478	2008-04-08	12
-298	val_298	2008-04-08	12
-125	val_125	2008-04-08	12
-431	val_431	2008-04-08	12
-164	val_164	2008-04-08	12
-424	val_424	2008-04-08	12
-187	val_187	2008-04-08	12
-382	val_382	2008-04-08	12
-5	val_5	2008-04-08	12
-70	val_70	2008-04-08	12
-397	val_397	2008-04-08	12
-480	val_480	2008-04-08	12
-291	val_291	2008-04-08	12
-24	val_24	2008-04-08	12
-351	val_351	2008-04-08	12
-255	val_255	2008-04-08	12
-104	val_104	2008-04-08	12
-70	val_70	2008-04-08	12
-163	val_163	2008-04-08	12
-438	val_438	2008-04-08	12
-119	val_119	2008-04-08	12
-414	val_414	2008-04-08	12
-200	val_200	2008-04-08	12
-491	val_491	2008-04-08	12
-237	val_237	2008-04-08	12
-439	val_439	2008-04-08	12
-360	val_360	2008-04-08	12
-248	val_248	2008-04-08	12
-479	val_479	2008-04-08	12
-305	val_305	2008-04-08	12
-417	val_417	2008-04-08	12
-199	val_199	2008-04-08	12
-444	val_444	2008-04-08	12
-120	val_120	2008-04-08	12
-429	val_429	2008-04-08	12
-169	val_169	2008-04-08	12
-443	val_443	2008-04-08	12
-323	val_323	2008-04-08	12
-325	val_325	2008-04-08	12
-277	val_277	2008-04-08	12
-230	val_230	2008-04-08	12
-478	val_478	2008-04-08	12
-178	val_178	2008-04-08	12
-468	val_468	2008-04-08	12
-310	val_310	2008-04-08	12
-317	val_317	2008-04-08	12
-333	val_333	2008-04-08	12
-493	val_493	2008-04-08	12
-460	val_460	2008-04-08	12
-207	val_207	2008-04-08	12
-249	val_249	2008-04-08	12
-265	val_265	2008-04-08	12
-480	val_480	2008-04-08	12
-83	val_83	2008-04-08	12
-136	val_136	2008-04-08	12
-353	val_353	2008-04-08	12
-172	val_172	2008-04-08	12
-214	val_214	2008-04-08	12
-462	val_462	2008-04-08	12
-233	val_233	2008-04-08	12
-406	val_406	2008-04-08	12
-133	val_133	2008-04-08	12
-175	val_175	2008-04-08	12
-189	val_189	2008-04-08	12
-454	val_454	2008-04-08	12
-375	val_375	2008-04-08	12
-401	val_401	2008-04-08	12
-421	val_421	2008-04-08	12
-407	val_407	2008-04-08	12
-384	val_384	2008-04-08	12
-256	val_256	2008-04-08	12
-26	val_26	2008-04-08	12
-134	val_134	2008-04-08	12
-67	val_67	2008-04-08	12
-384	val_384	2008-04-08	12
-379	val_379	2008-04-08	12
-18	val_18	2008-04-08	12
-462	val_462	2008-04-08	12
-492	val_492	2008-04-08	12
-100	val_100	2008-04-08	12
-298	val_298	2008-04-08	12
-9	val_9	2008-04-08	12
-341	val_341	2008-04-08	12
-498	val_498	2008-04-08	12
-146	val_146	2008-04-08	12
-458	val_458	2008-04-08	12
-362	val_362	2008-04-08	12
-186	val_186	2008-04-08	12
-285	val_285	2008-04-08	12
-348	val_348	2008-04-08	12
-167	val_167	2008-04-08	12
-18	val_18	2008-04-08	12
-273	val_273	2008-04-08	12
-183	val_183	2008-04-08	12
-281	val_281	2008-04-08	12
-344	val_344	2008-04-08	12
-97	val_97	2008-04-08	12
-469	val_469	2008-04-08	12
-315	val_315	2008-04-08	12
-84	val_84	2008-04-08	12
-28	val_28	2008-04-08	12
-37	val_37	2008-04-08	12
-448	val_448	2008-04-08	12
-152	val_152	2008-04-08	12
-348	val_348	2008-04-08	12
-307	val_307	2008-04-08	12
-194	val_194	2008-04-08	12
-414	val_414	2008-04-08	12
-477	val_477	2008-04-08	12
-222	val_222	2008-04-08	12
-126	val_126	2008-04-08	12
-90	val_90	2008-04-08	12
-169	val_169	2008-04-08	12
-403	val_403	2008-04-08	12
-400	val_400	2008-04-08	12
-200	val_200	2008-04-08	12
-97	val_97	2008-04-08	12
diff --git a/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 b/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 b/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 b/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f b/sql/hive/src/test/resources/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/load_file_with_space_in_the_name-1-d19201e2fcaee4d451292bd740e6c637
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc
rename to sql/hive/src/test/resources/golden/load_file_with_space_in_the_name-1-d19201e2fcaee4d451292bd740e6c637
diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/load_file_with_space_in_the_name-2-ad8795e50f5998ea1d2eb64a0c02e6e5
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a
rename to sql/hive/src/test/resources/golden/load_file_with_space_in_the_name-2-ad8795e50f5998ea1d2eb64a0c02e6e5
diff --git a/sql/hive/src/test/resources/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97 b/sql/hive/src/test/resources/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731 b/sql/hive/src/test/resources/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 b/sql/hive/src/test/resources/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc b/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 b/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450 b/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 b/sql/hive/src/test/resources/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 b/sql/hive/src/test/resources/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 b/sql/hive/src/test/resources/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 b/sql/hive/src/test/resources/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c b/sql/hive/src/test/resources/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/loadpart1-1-6cc94d19c536a996592629f7c82c2ac9
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394
rename to sql/hive/src/test/resources/golden/loadpart1-1-6cc94d19c536a996592629f7c82c2ac9
diff --git a/sql/hive/src/test/resources/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 b/sql/hive/src/test/resources/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9
index cf367dacbdf0ed26fa27a324e2e377c0c11e7759..bf2a7f452edc3b08fdad3f9f8cafc91ede4a2534 100644
--- a/sql/hive/src/test/resources/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9
+++ b/sql/hive/src/test/resources/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9
@@ -3,4 +3,4 @@
 3	test_part	test_Part
 4	test_part	test_Part
 5	test_part	test_Part
-6	test_part	test_Part
\ No newline at end of file
+6	test_part	test_Part
diff --git a/sql/hive/src/test/resources/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 b/sql/hive/src/test/resources/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4
index cf367dacbdf0ed26fa27a324e2e377c0c11e7759..bf2a7f452edc3b08fdad3f9f8cafc91ede4a2534 100644
--- a/sql/hive/src/test/resources/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4
+++ b/sql/hive/src/test/resources/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4
@@ -3,4 +3,4 @@
 3	test_part	test_Part
 4	test_part	test_Part
 5	test_part	test_Part
-6	test_part	test_Part
\ No newline at end of file
+6	test_part	test_Part
diff --git a/sql/hive/src/test/resources/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e b/sql/hive/src/test/resources/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock1-1-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/lock1-1-3e95421993ab28d18245ec2340f580a3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock1-2-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/lock1-2-c0c18ac884677231a41eea8d980d0451
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock2-1-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/lock2-1-3e95421993ab28d18245ec2340f580a3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock2-2-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/lock2-2-c0c18ac884677231a41eea8d980d0451
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock2-3-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/lock2-3-27ad2962fed131f51ba802596ba37278
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a b/sql/hive/src/test/resources/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca b/sql/hive/src/test/resources/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock3-0-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/lock3-0-27ad2962fed131f51ba802596ba37278
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a b/sql/hive/src/test/resources/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock3-2-b1ca816784e88f105b2fce1175340c33 b/sql/hive/src/test/resources/golden/lock3-2-b1ca816784e88f105b2fce1175340c33
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac b/sql/hive/src/test/resources/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe b/sql/hive/src/test/resources/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock4-1-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/lock4-1-27ad2962fed131f51ba802596ba37278
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a b/sql/hive/src/test/resources/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock4-3-b1ca816784e88f105b2fce1175340c33 b/sql/hive/src/test/resources/golden/lock4-3-b1ca816784e88f105b2fce1175340c33
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac b/sql/hive/src/test/resources/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe b/sql/hive/src/test/resources/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73 b/sql/hive/src/test/resources/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73
index 19492fd335bcb6752ca9d2dfe37e21d1abf8362d..a86e9c5af723e131f574dc4c9b7901c3ac5641f7 100644
--- a/sql/hive/src/test/resources/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73
+++ b/sql/hive/src/test/resources/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73
@@ -9,4 +9,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef b/sql/hive/src/test/resources/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef
index 3a57720041fb345fec33000d5b7cbf8005c697db..d091388cd5e19263f1797a7e6cff6e73e25ea2b0 100644
--- a/sql/hive/src/test/resources/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef
+++ b/sql/hive/src/test/resources/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef
@@ -9,4 +9,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 b/sql/hive/src/test/resources/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735
index 19492fd335bcb6752ca9d2dfe37e21d1abf8362d..a86e9c5af723e131f574dc4c9b7901c3ac5641f7 100644
--- a/sql/hive/src/test/resources/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735
+++ b/sql/hive/src/test/resources/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735
@@ -9,4 +9,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703 b/sql/hive/src/test/resources/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703
index 3a57720041fb345fec33000d5b7cbf8005c697db..d091388cd5e19263f1797a7e6cff6e73e25ea2b0 100644
--- a/sql/hive/src/test/resources/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703
+++ b/sql/hive/src/test/resources/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703
@@ -9,4 +9,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d b/sql/hive/src/test/resources/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d
deleted file mode 100644
index 657eea30d475f38c9dc048671dc1beb6cf50e647..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d
+++ /dev/null
@@ -1 +0,0 @@
-1114788
diff --git a/sql/hive/src/test/resources/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768 b/sql/hive/src/test/resources/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 b/sql/hive/src/test/resources/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505
deleted file mode 100644
index cfae441c4a10a4a2122ca280b1d82e3aa766b7ee..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505
+++ /dev/null
@@ -1,10 +0,0 @@
-NULL	NULL	238	val_238
-86	val_86	86	val_86
-NULL	NULL	311	val_311
-27	val_27	27	val_27
-NULL	NULL	165	val_165
-NULL	NULL	409	val_409
-NULL	NULL	255	val_255
-NULL	NULL	278	val_278
-98	val_98	98	val_98
-98	val_98	98	val_98
diff --git a/sql/hive/src/test/resources/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 b/sql/hive/src/test/resources/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 b/sql/hive/src/test/resources/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9
deleted file mode 100644
index 6d8155efd76cd2341dacb78f4355d70069377940..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9
+++ /dev/null
@@ -1,10 +0,0 @@
-238	val_238	238	{"key":238,"value":"val_238"}
-238	val_238	238	{"key":238,"value":"val_238"}
-NULL	NULL	86	{"key":86,"value":"val_86"}
-311	val_311	311	{"key":311,"value":"val_311"}
-311	val_311	311	{"key":311,"value":"val_311"}
-311	val_311	311	{"key":311,"value":"val_311"}
-NULL	NULL	27	{"key":27,"value":"val_27"}
-NULL	NULL	165	{"key":165,"value":"val_165"}
-409	val_409	409	{"key":409,"value":"val_409"}
-409	val_409	409	{"key":409,"value":"val_409"}
diff --git a/sql/hive/src/test/resources/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 b/sql/hive/src/test/resources/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0
deleted file mode 100644
index 44f1acd59de68188dfaafb0f8b46039683874112..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0
+++ /dev/null
@@ -1,10 +0,0 @@
-238	val_238	238	val_238
-238	val_238	238	val_238
-86	val_86	86	val_86
-311	val_311	311	val_311
-311	val_311	311	val_311
-311	val_311	311	val_311
-27	val_27	27	val_27
-165	val_165	165	val_165
-165	val_165	165	val_165
-409	val_409	409	val_409
diff --git a/sql/hive/src/test/resources/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d b/sql/hive/src/test/resources/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 b/sql/hive/src/test/resources/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505
deleted file mode 100644
index cfae441c4a10a4a2122ca280b1d82e3aa766b7ee..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505
+++ /dev/null
@@ -1,10 +0,0 @@
-NULL	NULL	238	val_238
-86	val_86	86	val_86
-NULL	NULL	311	val_311
-27	val_27	27	val_27
-NULL	NULL	165	val_165
-NULL	NULL	409	val_409
-NULL	NULL	255	val_255
-NULL	NULL	278	val_278
-98	val_98	98	val_98
-98	val_98	98	val_98
diff --git a/sql/hive/src/test/resources/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba b/sql/hive/src/test/resources/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 b/sql/hive/src/test/resources/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9
deleted file mode 100644
index 6d8155efd76cd2341dacb78f4355d70069377940..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9
+++ /dev/null
@@ -1,10 +0,0 @@
-238	val_238	238	{"key":238,"value":"val_238"}
-238	val_238	238	{"key":238,"value":"val_238"}
-NULL	NULL	86	{"key":86,"value":"val_86"}
-311	val_311	311	{"key":311,"value":"val_311"}
-311	val_311	311	{"key":311,"value":"val_311"}
-311	val_311	311	{"key":311,"value":"val_311"}
-NULL	NULL	27	{"key":27,"value":"val_27"}
-NULL	NULL	165	{"key":165,"value":"val_165"}
-409	val_409	409	{"key":409,"value":"val_409"}
-409	val_409	409	{"key":409,"value":"val_409"}
diff --git a/sql/hive/src/test/resources/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 b/sql/hive/src/test/resources/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 b/sql/hive/src/test/resources/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0
deleted file mode 100644
index 44f1acd59de68188dfaafb0f8b46039683874112..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0
+++ /dev/null
@@ -1,10 +0,0 @@
-238	val_238	238	val_238
-238	val_238	238	val_238
-86	val_86	86	val_86
-311	val_311	311	val_311
-311	val_311	311	val_311
-311	val_311	311	val_311
-27	val_27	27	val_27
-165	val_165	165	val_165
-165	val_165	165	val_165
-409	val_409	409	val_409
diff --git a/sql/hive/src/test/resources/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 b/sql/hive/src/test/resources/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52
+++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257
+++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f b/sql/hive/src/test/resources/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f
index eab7fd7a51ea417943aaa3ac8aeadf46203c6fad..36b4fccb585a8c8599344f63937d0ea41bdaeecf 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f
+++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f
@@ -7,4 +7,4 @@ val_105
 val_11
 val_111
 val_113
-val_114
\ No newline at end of file
+val_114
diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291
+++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e
+++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f b/sql/hive/src/test/resources/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f
index eab7fd7a51ea417943aaa3ac8aeadf46203c6fad..36b4fccb585a8c8599344f63937d0ea41bdaeecf 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f
+++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f
@@ -7,4 +7,4 @@ val_105
 val_11
 val_111
 val_113
-val_114
\ No newline at end of file
+val_114
diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f b/sql/hive/src/test/resources/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f
index eab7fd7a51ea417943aaa3ac8aeadf46203c6fad..36b4fccb585a8c8599344f63937d0ea41bdaeecf 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f
+++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f
@@ -7,4 +7,4 @@ val_105
 val_11
 val_111
 val_113
-val_114
\ No newline at end of file
+val_114
diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52
+++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e
+++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f b/sql/hive/src/test/resources/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f
index eab7fd7a51ea417943aaa3ac8aeadf46203c6fad..36b4fccb585a8c8599344f63937d0ea41bdaeecf 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f
+++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f
@@ -7,4 +7,4 @@ val_105
 val_11
 val_111
 val_113
-val_114
\ No newline at end of file
+val_114
diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291
+++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257
+++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751
+++ b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-4-5166a5b9d30dfacbe33dd909c0df6310
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1
rename to sql/hive/src/test/resources/golden/mapjoin_mapjoin-4-5166a5b9d30dfacbe33dd909c0df6310
diff --git a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-5-5ee898cab883074f3297198c52445ee4 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-5-5ee898cab883074f3297198c52445ee4
new file mode 100644
index 0000000000000000000000000000000000000000..0588d12b85bd3a6cdd1e8501a007ea4b133e25ae
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-5-5ee898cab883074f3297198c52445ee4
@@ -0,0 +1,20 @@
+98
+98
+66
+98
+98
+98
+98
+66
+98
+98
+98
+98
+66
+98
+98
+98
+98
+66
+98
+98
diff --git a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7
deleted file mode 100644
index 8e7fe1e4cae08635dd51b830164941d15398180c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7
+++ /dev/null
@@ -1,2 +0,0 @@
-5308
-5308
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-6-dca8c08a235b45d1cdcb94e363afb17
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77
rename to sql/hive/src/test/resources/golden/mapjoin_mapjoin-6-dca8c08a235b45d1cdcb94e363afb17
diff --git a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-7-fddbdea343a9ddb5f8dedc18147640b7 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-7-fddbdea343a9ddb5f8dedc18147640b7
new file mode 100644
index 0000000000000000000000000000000000000000..293e1c3f1edcae0a86da8b6761ab5e4d4acf8484
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-7-fddbdea343a9ddb5f8dedc18147640b7
@@ -0,0 +1,2 @@
+5308
+5308
diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-8-2be637ed4f6146e8525ae1a863e72736
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/mapjoin_mapjoin-8-2be637ed4f6146e8525ae1a863e72736
diff --git a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-9-c47698bac140454637a999e583941ce7 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-9-c47698bac140454637a999e583941ce7
new file mode 100644
index 0000000000000000000000000000000000000000..293e1c3f1edcae0a86da8b6761ab5e4d4acf8484
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-9-c47698bac140454637a999e583941ce7
@@ -0,0 +1,2 @@
+5308
+5308
diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751
+++ b/sql/hive/src/test/resources/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924 b/sql/hive/src/test/resources/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924
index 482848c1ef8aad54af40dc15f1a7839d389d71b8..89ea689f2a33b7e7940d5c9555e6d2d4038a4a5c 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924
+++ b/sql/hive/src/test/resources/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924
@@ -104,4 +104,4 @@
 406	val_406
 406	val_406
 406	val_406
-406	val_406
\ No newline at end of file
+406	val_406
diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b b/sql/hive/src/test/resources/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b
index 482848c1ef8aad54af40dc15f1a7839d389d71b8..89ea689f2a33b7e7940d5c9555e6d2d4038a4a5c 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b
+++ b/sql/hive/src/test/resources/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b
@@ -104,4 +104,4 @@
 406	val_406
 406	val_406
 406	val_406
-406	val_406
\ No newline at end of file
+406	val_406
diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751
+++ b/sql/hive/src/test/resources/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951
index 5c62121a6600c50f68a3a1e7ebd4c08a8a3e4ca7..7b3f58fdde1af9fc23cc69e8a44d6aa77013ceeb 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951
+++ b/sql/hive/src/test/resources/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951
@@ -1,2 +1,2 @@
 2	Joe	2	Tie	2	Tie
-2	Hank	2	Tie	2	Tie
\ No newline at end of file
+2	Hank	2	Tie	2	Tie
diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/mapjoin_subquery2-6-9bf06af695892b0d7067d5b30e0b2425
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd
rename to sql/hive/src/test/resources/golden/mapjoin_subquery2-6-9bf06af695892b0d7067d5b30e0b2425
diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-7-c6b0cdb137f13f8362c0c49c544151a4
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/mapjoin_subquery2-7-c6b0cdb137f13f8362c0c49c544151a4
diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/mapjoin_subquery2-8-131ae5ecfff2733b04bdfada0108cf40
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc
rename to sql/hive/src/test/resources/golden/mapjoin_subquery2-8-131ae5ecfff2733b04bdfada0108cf40
diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e b/sql/hive/src/test/resources/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d b/sql/hive/src/test/resources/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d
+++ b/sql/hive/src/test/resources/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a
+++ b/sql/hive/src/test/resources/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795
index 271f3e168fd2391b9ad56277dda387d2f6d05dbc..984f4f37f676fdbb2df8373b60fd5900af698e3b 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795
+++ b/sql/hive/src/test/resources/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795
@@ -1,4 +1,4 @@
 128		128		128	
 146	val_146	146	val_146	146	val_146
 224		224		224	
-369		369		369	
\ No newline at end of file
+369		369		369	
diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420
index 271f3e168fd2391b9ad56277dda387d2f6d05dbc..984f4f37f676fdbb2df8373b60fd5900af698e3b 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420
+++ b/sql/hive/src/test/resources/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420
@@ -1,4 +1,4 @@
 128		128		128	
 146	val_146	146	val_146	146	val_146
 224		224		224	
-369		369		369	
\ No newline at end of file
+369		369		369	
diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906
index f00666f6113d2447c4a45a9b37e7a2c09f6e18ef..ece00d994625779f26df07e6dc36718882a588f7 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906
+++ b/sql/hive/src/test/resources/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906
@@ -2,4 +2,4 @@ NULL	NULL	333444	555666	333444	555666
 128		128		128	
 146	val_146	146	val_146	146	val_146
 224		224		224	
-369		369		369	
\ No newline at end of file
+369		369		369	
diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906
index f00666f6113d2447c4a45a9b37e7a2c09f6e18ef..ece00d994625779f26df07e6dc36718882a588f7 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906
+++ b/sql/hive/src/test/resources/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906
@@ -2,4 +2,4 @@ NULL	NULL	333444	555666	333444	555666
 128		128		128	
 146	val_146	146	val_146	146	val_146
 224		224		224	
-369		369		369	
\ No newline at end of file
+369		369		369	
diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a b/sql/hive/src/test/resources/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a
index f00666f6113d2447c4a45a9b37e7a2c09f6e18ef..ece00d994625779f26df07e6dc36718882a588f7 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a
+++ b/sql/hive/src/test/resources/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a
@@ -2,4 +2,4 @@ NULL	NULL	333444	555666	333444	555666
 128		128		128	
 146	val_146	146	val_146	146	val_146
 224		224		224	
-369		369		369	
\ No newline at end of file
+369		369		369	
diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017
index 271f3e168fd2391b9ad56277dda387d2f6d05dbc..984f4f37f676fdbb2df8373b60fd5900af698e3b 100644
--- a/sql/hive/src/test/resources/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017
+++ b/sql/hive/src/test/resources/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017
@@ -1,4 +1,4 @@
 128		128		128	
 146	val_146	146	val_146	146	val_146
 224		224		224	
-369		369		369	
\ No newline at end of file
+369		369		369	
diff --git a/sql/hive/src/test/resources/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc b/sql/hive/src/test/resources/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc
index 4ab18ace38f46252ac51b8425173970ae02e4e6a..777c07c766030205faac44c0c22da57986eaa9fb 100644
--- a/sql/hive/src/test/resources/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc
+++ b/sql/hive/src/test/resources/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc
@@ -497,4 +497,4 @@
 497	49	7	val_497
 498	49	8	val_498
 498	49	8	val_498
-498	49	8	val_498
\ No newline at end of file
+498	49	8	val_498
diff --git a/sql/hive/src/test/resources/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f
index 1504c19f78eb288c2d9dadc95ed8f9545520a422..5aa1d6b005a24a7e3ccb71aaa890488b8fc20d7d 100644
--- a/sql/hive/src/test/resources/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -497,4 +497,4 @@
 97	9	7	val_97
 97	9	7	val_97
 98	9	8	val_98
-98	9	8	val_98
\ No newline at end of file
+98	9	8	val_98
diff --git a/sql/hive/src/test/resources/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027
+++ b/sql/hive/src/test/resources/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138 b/sql/hive/src/test/resources/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138
+++ b/sql/hive/src/test/resources/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/merge1-4-3277fe538b66923cd879b45371838d2b b/sql/hive/src/test/resources/golden/merge1-4-3277fe538b66923cd879b45371838d2b
index df07a9da29f01512947648bed0e2a4f71e8798dc..93e965c7714038926c1e1173addaeed83556567f 100644
--- a/sql/hive/src/test/resources/golden/merge1-4-3277fe538b66923cd879b45371838d2b
+++ b/sql/hive/src/test/resources/golden/merge1-4-3277fe538b66923cd879b45371838d2b
@@ -306,4 +306,4 @@
 495	1
 496	1
 497	1
-498	3
\ No newline at end of file
+498	3
diff --git a/sql/hive/src/test/resources/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a
+++ b/sql/hive/src/test/resources/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027
+++ b/sql/hive/src/test/resources/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138 b/sql/hive/src/test/resources/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138
+++ b/sql/hive/src/test/resources/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf
+++ b/sql/hive/src/test/resources/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86
+++ b/sql/hive/src/test/resources/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076
+++ b/sql/hive/src/test/resources/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea
+++ b/sql/hive/src/test/resources/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd b/sql/hive/src/test/resources/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd
index df07a9da29f01512947648bed0e2a4f71e8798dc..93e965c7714038926c1e1173addaeed83556567f 100644
--- a/sql/hive/src/test/resources/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd
+++ b/sql/hive/src/test/resources/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd
@@ -306,4 +306,4 @@
 495	1
 496	1
 497	1
-498	3
\ No newline at end of file
+498	3
diff --git a/sql/hive/src/test/resources/golden/merge4-10-692a197bd688b48f762e72978f54aa32 b/sql/hive/src/test/resources/golden/merge4-10-692a197bd688b48f762e72978f54aa32
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 b/sql/hive/src/test/resources/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868
deleted file mode 100644
index 67c6db85915498e343534ff94e7efed5f3faa0eb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868
+++ /dev/null
@@ -1,1500 +0,0 @@
-238	val_238	2010-08-15	11
-86	val_86	2010-08-15	11
-311	val_311	2010-08-15	11
-27	val_27	2010-08-15	11
-165	val_165	2010-08-15	11
-409	val_409	2010-08-15	11
-255	val_255	2010-08-15	11
-278	val_278	2010-08-15	11
-98	val_98	2010-08-15	11
-484	val_484	2010-08-15	11
-265	val_265	2010-08-15	11
-193	val_193	2010-08-15	11
-401	val_401	2010-08-15	11
-150	val_150	2010-08-15	11
-273	val_273	2010-08-15	11
-224	val_224	2010-08-15	11
-369	val_369	2010-08-15	11
-66	val_66	2010-08-15	11
-128	val_128	2010-08-15	11
-213	val_213	2010-08-15	11
-146	val_146	2010-08-15	11
-406	val_406	2010-08-15	11
-429	val_429	2010-08-15	11
-374	val_374	2010-08-15	11
-152	val_152	2010-08-15	11
-469	val_469	2010-08-15	11
-145	val_145	2010-08-15	11
-495	val_495	2010-08-15	11
-37	val_37	2010-08-15	11
-327	val_327	2010-08-15	11
-281	val_281	2010-08-15	11
-277	val_277	2010-08-15	11
-209	val_209	2010-08-15	11
-15	val_15	2010-08-15	11
-82	val_82	2010-08-15	11
-403	val_403	2010-08-15	11
-166	val_166	2010-08-15	11
-417	val_417	2010-08-15	11
-430	val_430	2010-08-15	11
-252	val_252	2010-08-15	11
-292	val_292	2010-08-15	11
-219	val_219	2010-08-15	11
-287	val_287	2010-08-15	11
-153	val_153	2010-08-15	11
-193	val_193	2010-08-15	11
-338	val_338	2010-08-15	11
-446	val_446	2010-08-15	11
-459	val_459	2010-08-15	11
-394	val_394	2010-08-15	11
-237	val_237	2010-08-15	11
-482	val_482	2010-08-15	11
-174	val_174	2010-08-15	11
-413	val_413	2010-08-15	11
-494	val_494	2010-08-15	11
-207	val_207	2010-08-15	11
-199	val_199	2010-08-15	11
-466	val_466	2010-08-15	11
-208	val_208	2010-08-15	11
-174	val_174	2010-08-15	11
-399	val_399	2010-08-15	11
-396	val_396	2010-08-15	11
-247	val_247	2010-08-15	11
-417	val_417	2010-08-15	11
-489	val_489	2010-08-15	11
-162	val_162	2010-08-15	11
-377	val_377	2010-08-15	11
-397	val_397	2010-08-15	11
-309	val_309	2010-08-15	11
-365	val_365	2010-08-15	11
-266	val_266	2010-08-15	11
-439	val_439	2010-08-15	11
-342	val_342	2010-08-15	11
-367	val_367	2010-08-15	11
-325	val_325	2010-08-15	11
-167	val_167	2010-08-15	11
-195	val_195	2010-08-15	11
-475	val_475	2010-08-15	11
-17	val_17	2010-08-15	11
-113	val_113	2010-08-15	11
-155	val_155	2010-08-15	11
-203	val_203	2010-08-15	11
-339	val_339	2010-08-15	11
-0	val_0	2010-08-15	11
-455	val_455	2010-08-15	11
-128	val_128	2010-08-15	11
-311	val_311	2010-08-15	11
-316	val_316	2010-08-15	11
-57	val_57	2010-08-15	11
-302	val_302	2010-08-15	11
-205	val_205	2010-08-15	11
-149	val_149	2010-08-15	11
-438	val_438	2010-08-15	11
-345	val_345	2010-08-15	11
-129	val_129	2010-08-15	11
-170	val_170	2010-08-15	11
-20	val_20	2010-08-15	11
-489	val_489	2010-08-15	11
-157	val_157	2010-08-15	11
-378	val_378	2010-08-15	11
-221	val_221	2010-08-15	11
-92	val_92	2010-08-15	11
-111	val_111	2010-08-15	11
-47	val_47	2010-08-15	11
-72	val_72	2010-08-15	11
-4	val_4	2010-08-15	11
-280	val_280	2010-08-15	11
-35	val_35	2010-08-15	11
-427	val_427	2010-08-15	11
-277	val_277	2010-08-15	11
-208	val_208	2010-08-15	11
-356	val_356	2010-08-15	11
-399	val_399	2010-08-15	11
-169	val_169	2010-08-15	11
-382	val_382	2010-08-15	11
-498	val_498	2010-08-15	11
-125	val_125	2010-08-15	11
-386	val_386	2010-08-15	11
-437	val_437	2010-08-15	11
-469	val_469	2010-08-15	11
-192	val_192	2010-08-15	11
-286	val_286	2010-08-15	11
-187	val_187	2010-08-15	11
-176	val_176	2010-08-15	11
-54	val_54	2010-08-15	11
-459	val_459	2010-08-15	11
-51	val_51	2010-08-15	11
-138	val_138	2010-08-15	11
-103	val_103	2010-08-15	11
-239	val_239	2010-08-15	11
-213	val_213	2010-08-15	11
-216	val_216	2010-08-15	11
-430	val_430	2010-08-15	11
-278	val_278	2010-08-15	11
-176	val_176	2010-08-15	11
-289	val_289	2010-08-15	11
-221	val_221	2010-08-15	11
-65	val_65	2010-08-15	11
-318	val_318	2010-08-15	11
-332	val_332	2010-08-15	11
-311	val_311	2010-08-15	11
-275	val_275	2010-08-15	11
-137	val_137	2010-08-15	11
-241	val_241	2010-08-15	11
-83	val_83	2010-08-15	11
-333	val_333	2010-08-15	11
-180	val_180	2010-08-15	11
-284	val_284	2010-08-15	11
-12	val_12	2010-08-15	11
-230	val_230	2010-08-15	11
-181	val_181	2010-08-15	11
-67	val_67	2010-08-15	11
-260	val_260	2010-08-15	11
-404	val_404	2010-08-15	11
-384	val_384	2010-08-15	11
-489	val_489	2010-08-15	11
-353	val_353	2010-08-15	11
-373	val_373	2010-08-15	11
-272	val_272	2010-08-15	11
-138	val_138	2010-08-15	11
-217	val_217	2010-08-15	11
-84	val_84	2010-08-15	11
-348	val_348	2010-08-15	11
-466	val_466	2010-08-15	11
-58	val_58	2010-08-15	11
-8	val_8	2010-08-15	11
-411	val_411	2010-08-15	11
-230	val_230	2010-08-15	11
-208	val_208	2010-08-15	11
-348	val_348	2010-08-15	11
-24	val_24	2010-08-15	11
-463	val_463	2010-08-15	11
-431	val_431	2010-08-15	11
-179	val_179	2010-08-15	11
-172	val_172	2010-08-15	11
-42	val_42	2010-08-15	11
-129	val_129	2010-08-15	11
-158	val_158	2010-08-15	11
-119	val_119	2010-08-15	11
-496	val_496	2010-08-15	11
-0	val_0	2010-08-15	11
-322	val_322	2010-08-15	11
-197	val_197	2010-08-15	11
-468	val_468	2010-08-15	11
-393	val_393	2010-08-15	11
-454	val_454	2010-08-15	11
-100	val_100	2010-08-15	11
-298	val_298	2010-08-15	11
-199	val_199	2010-08-15	11
-191	val_191	2010-08-15	11
-418	val_418	2010-08-15	11
-96	val_96	2010-08-15	11
-26	val_26	2010-08-15	11
-165	val_165	2010-08-15	11
-327	val_327	2010-08-15	11
-230	val_230	2010-08-15	11
-205	val_205	2010-08-15	11
-120	val_120	2010-08-15	11
-131	val_131	2010-08-15	11
-51	val_51	2010-08-15	11
-404	val_404	2010-08-15	11
-43	val_43	2010-08-15	11
-436	val_436	2010-08-15	11
-156	val_156	2010-08-15	11
-469	val_469	2010-08-15	11
-468	val_468	2010-08-15	11
-308	val_308	2010-08-15	11
-95	val_95	2010-08-15	11
-196	val_196	2010-08-15	11
-288	val_288	2010-08-15	11
-481	val_481	2010-08-15	11
-457	val_457	2010-08-15	11
-98	val_98	2010-08-15	11
-282	val_282	2010-08-15	11
-197	val_197	2010-08-15	11
-187	val_187	2010-08-15	11
-318	val_318	2010-08-15	11
-318	val_318	2010-08-15	11
-409	val_409	2010-08-15	11
-470	val_470	2010-08-15	11
-137	val_137	2010-08-15	11
-369	val_369	2010-08-15	11
-316	val_316	2010-08-15	11
-169	val_169	2010-08-15	11
-413	val_413	2010-08-15	11
-85	val_85	2010-08-15	11
-77	val_77	2010-08-15	11
-0	val_0	2010-08-15	11
-490	val_490	2010-08-15	11
-87	val_87	2010-08-15	11
-364	val_364	2010-08-15	11
-179	val_179	2010-08-15	11
-118	val_118	2010-08-15	11
-134	val_134	2010-08-15	11
-395	val_395	2010-08-15	11
-282	val_282	2010-08-15	11
-138	val_138	2010-08-15	11
-238	val_238	2010-08-15	11
-419	val_419	2010-08-15	11
-15	val_15	2010-08-15	11
-118	val_118	2010-08-15	11
-72	val_72	2010-08-15	11
-90	val_90	2010-08-15	11
-307	val_307	2010-08-15	11
-19	val_19	2010-08-15	11
-435	val_435	2010-08-15	11
-10	val_10	2010-08-15	11
-277	val_277	2010-08-15	11
-273	val_273	2010-08-15	11
-306	val_306	2010-08-15	11
-224	val_224	2010-08-15	11
-309	val_309	2010-08-15	11
-389	val_389	2010-08-15	11
-327	val_327	2010-08-15	11
-242	val_242	2010-08-15	11
-369	val_369	2010-08-15	11
-392	val_392	2010-08-15	11
-272	val_272	2010-08-15	11
-331	val_331	2010-08-15	11
-401	val_401	2010-08-15	11
-242	val_242	2010-08-15	11
-452	val_452	2010-08-15	11
-177	val_177	2010-08-15	11
-226	val_226	2010-08-15	11
-5	val_5	2010-08-15	11
-497	val_497	2010-08-15	11
-402	val_402	2010-08-15	11
-396	val_396	2010-08-15	11
-317	val_317	2010-08-15	11
-395	val_395	2010-08-15	11
-58	val_58	2010-08-15	11
-35	val_35	2010-08-15	11
-336	val_336	2010-08-15	11
-95	val_95	2010-08-15	11
-11	val_11	2010-08-15	11
-168	val_168	2010-08-15	11
-34	val_34	2010-08-15	11
-229	val_229	2010-08-15	11
-233	val_233	2010-08-15	11
-143	val_143	2010-08-15	11
-472	val_472	2010-08-15	11
-322	val_322	2010-08-15	11
-498	val_498	2010-08-15	11
-160	val_160	2010-08-15	11
-195	val_195	2010-08-15	11
-42	val_42	2010-08-15	11
-321	val_321	2010-08-15	11
-430	val_430	2010-08-15	11
-119	val_119	2010-08-15	11
-489	val_489	2010-08-15	11
-458	val_458	2010-08-15	11
-78	val_78	2010-08-15	11
-76	val_76	2010-08-15	11
-41	val_41	2010-08-15	11
-223	val_223	2010-08-15	11
-492	val_492	2010-08-15	11
-149	val_149	2010-08-15	11
-449	val_449	2010-08-15	11
-218	val_218	2010-08-15	11
-228	val_228	2010-08-15	11
-138	val_138	2010-08-15	11
-453	val_453	2010-08-15	11
-30	val_30	2010-08-15	11
-209	val_209	2010-08-15	11
-64	val_64	2010-08-15	11
-468	val_468	2010-08-15	11
-76	val_76	2010-08-15	11
-74	val_74	2010-08-15	11
-342	val_342	2010-08-15	11
-69	val_69	2010-08-15	11
-230	val_230	2010-08-15	11
-33	val_33	2010-08-15	11
-368	val_368	2010-08-15	11
-103	val_103	2010-08-15	11
-296	val_296	2010-08-15	11
-113	val_113	2010-08-15	11
-216	val_216	2010-08-15	11
-367	val_367	2010-08-15	11
-344	val_344	2010-08-15	11
-167	val_167	2010-08-15	11
-274	val_274	2010-08-15	11
-219	val_219	2010-08-15	11
-239	val_239	2010-08-15	11
-485	val_485	2010-08-15	11
-116	val_116	2010-08-15	11
-223	val_223	2010-08-15	11
-256	val_256	2010-08-15	11
-263	val_263	2010-08-15	11
-70	val_70	2010-08-15	11
-487	val_487	2010-08-15	11
-480	val_480	2010-08-15	11
-401	val_401	2010-08-15	11
-288	val_288	2010-08-15	11
-191	val_191	2010-08-15	11
-5	val_5	2010-08-15	11
-244	val_244	2010-08-15	11
-438	val_438	2010-08-15	11
-128	val_128	2010-08-15	11
-467	val_467	2010-08-15	11
-432	val_432	2010-08-15	11
-202	val_202	2010-08-15	11
-316	val_316	2010-08-15	11
-229	val_229	2010-08-15	11
-469	val_469	2010-08-15	11
-463	val_463	2010-08-15	11
-280	val_280	2010-08-15	11
-2	val_2	2010-08-15	11
-35	val_35	2010-08-15	11
-283	val_283	2010-08-15	11
-331	val_331	2010-08-15	11
-235	val_235	2010-08-15	11
-80	val_80	2010-08-15	11
-44	val_44	2010-08-15	11
-193	val_193	2010-08-15	11
-321	val_321	2010-08-15	11
-335	val_335	2010-08-15	11
-104	val_104	2010-08-15	11
-466	val_466	2010-08-15	11
-366	val_366	2010-08-15	11
-175	val_175	2010-08-15	11
-403	val_403	2010-08-15	11
-483	val_483	2010-08-15	11
-53	val_53	2010-08-15	11
-105	val_105	2010-08-15	11
-257	val_257	2010-08-15	11
-406	val_406	2010-08-15	11
-409	val_409	2010-08-15	11
-190	val_190	2010-08-15	11
-406	val_406	2010-08-15	11
-401	val_401	2010-08-15	11
-114	val_114	2010-08-15	11
-258	val_258	2010-08-15	11
-90	val_90	2010-08-15	11
-203	val_203	2010-08-15	11
-262	val_262	2010-08-15	11
-348	val_348	2010-08-15	11
-424	val_424	2010-08-15	11
-12	val_12	2010-08-15	11
-396	val_396	2010-08-15	11
-201	val_201	2010-08-15	11
-217	val_217	2010-08-15	11
-164	val_164	2010-08-15	11
-431	val_431	2010-08-15	11
-454	val_454	2010-08-15	11
-478	val_478	2010-08-15	11
-298	val_298	2010-08-15	11
-125	val_125	2010-08-15	11
-431	val_431	2010-08-15	11
-164	val_164	2010-08-15	11
-424	val_424	2010-08-15	11
-187	val_187	2010-08-15	11
-382	val_382	2010-08-15	11
-5	val_5	2010-08-15	11
-70	val_70	2010-08-15	11
-397	val_397	2010-08-15	11
-480	val_480	2010-08-15	11
-291	val_291	2010-08-15	11
-24	val_24	2010-08-15	11
-351	val_351	2010-08-15	11
-255	val_255	2010-08-15	11
-104	val_104	2010-08-15	11
-70	val_70	2010-08-15	11
-163	val_163	2010-08-15	11
-438	val_438	2010-08-15	11
-119	val_119	2010-08-15	11
-414	val_414	2010-08-15	11
-200	val_200	2010-08-15	11
-491	val_491	2010-08-15	11
-237	val_237	2010-08-15	11
-439	val_439	2010-08-15	11
-360	val_360	2010-08-15	11
-248	val_248	2010-08-15	11
-479	val_479	2010-08-15	11
-305	val_305	2010-08-15	11
-417	val_417	2010-08-15	11
-199	val_199	2010-08-15	11
-444	val_444	2010-08-15	11
-120	val_120	2010-08-15	11
-429	val_429	2010-08-15	11
-169	val_169	2010-08-15	11
-443	val_443	2010-08-15	11
-323	val_323	2010-08-15	11
-325	val_325	2010-08-15	11
-277	val_277	2010-08-15	11
-230	val_230	2010-08-15	11
-478	val_478	2010-08-15	11
-178	val_178	2010-08-15	11
-468	val_468	2010-08-15	11
-310	val_310	2010-08-15	11
-317	val_317	2010-08-15	11
-333	val_333	2010-08-15	11
-493	val_493	2010-08-15	11
-460	val_460	2010-08-15	11
-207	val_207	2010-08-15	11
-249	val_249	2010-08-15	11
-265	val_265	2010-08-15	11
-480	val_480	2010-08-15	11
-83	val_83	2010-08-15	11
-136	val_136	2010-08-15	11
-353	val_353	2010-08-15	11
-172	val_172	2010-08-15	11
-214	val_214	2010-08-15	11
-462	val_462	2010-08-15	11
-233	val_233	2010-08-15	11
-406	val_406	2010-08-15	11
-133	val_133	2010-08-15	11
-175	val_175	2010-08-15	11
-189	val_189	2010-08-15	11
-454	val_454	2010-08-15	11
-375	val_375	2010-08-15	11
-401	val_401	2010-08-15	11
-421	val_421	2010-08-15	11
-407	val_407	2010-08-15	11
-384	val_384	2010-08-15	11
-256	val_256	2010-08-15	11
-26	val_26	2010-08-15	11
-134	val_134	2010-08-15	11
-67	val_67	2010-08-15	11
-384	val_384	2010-08-15	11
-379	val_379	2010-08-15	11
-18	val_18	2010-08-15	11
-462	val_462	2010-08-15	11
-492	val_492	2010-08-15	11
-100	val_100	2010-08-15	11
-298	val_298	2010-08-15	11
-9	val_9	2010-08-15	11
-341	val_341	2010-08-15	11
-498	val_498	2010-08-15	11
-146	val_146	2010-08-15	11
-458	val_458	2010-08-15	11
-362	val_362	2010-08-15	11
-186	val_186	2010-08-15	11
-285	val_285	2010-08-15	11
-348	val_348	2010-08-15	11
-167	val_167	2010-08-15	11
-18	val_18	2010-08-15	11
-273	val_273	2010-08-15	11
-183	val_183	2010-08-15	11
-281	val_281	2010-08-15	11
-344	val_344	2010-08-15	11
-97	val_97	2010-08-15	11
-469	val_469	2010-08-15	11
-315	val_315	2010-08-15	11
-84	val_84	2010-08-15	11
-28	val_28	2010-08-15	11
-37	val_37	2010-08-15	11
-448	val_448	2010-08-15	11
-152	val_152	2010-08-15	11
-348	val_348	2010-08-15	11
-307	val_307	2010-08-15	11
-194	val_194	2010-08-15	11
-414	val_414	2010-08-15	11
-477	val_477	2010-08-15	11
-222	val_222	2010-08-15	11
-126	val_126	2010-08-15	11
-90	val_90	2010-08-15	11
-169	val_169	2010-08-15	11
-403	val_403	2010-08-15	11
-400	val_400	2010-08-15	11
-200	val_200	2010-08-15	11
-97	val_97	2010-08-15	11
-238	val_238	2010-08-15	11
-86	val_86	2010-08-15	11
-311	val_311	2010-08-15	11
-27	val_27	2010-08-15	11
-165	val_165	2010-08-15	11
-409	val_409	2010-08-15	11
-255	val_255	2010-08-15	11
-278	val_278	2010-08-15	11
-98	val_98	2010-08-15	11
-484	val_484	2010-08-15	11
-265	val_265	2010-08-15	11
-193	val_193	2010-08-15	11
-401	val_401	2010-08-15	11
-150	val_150	2010-08-15	11
-273	val_273	2010-08-15	11
-224	val_224	2010-08-15	11
-369	val_369	2010-08-15	11
-66	val_66	2010-08-15	11
-128	val_128	2010-08-15	11
-213	val_213	2010-08-15	11
-146	val_146	2010-08-15	11
-406	val_406	2010-08-15	11
-429	val_429	2010-08-15	11
-374	val_374	2010-08-15	11
-152	val_152	2010-08-15	11
-469	val_469	2010-08-15	11
-145	val_145	2010-08-15	11
-495	val_495	2010-08-15	11
-37	val_37	2010-08-15	11
-327	val_327	2010-08-15	11
-281	val_281	2010-08-15	11
-277	val_277	2010-08-15	11
-209	val_209	2010-08-15	11
-15	val_15	2010-08-15	11
-82	val_82	2010-08-15	11
-403	val_403	2010-08-15	11
-166	val_166	2010-08-15	11
-417	val_417	2010-08-15	11
-430	val_430	2010-08-15	11
-252	val_252	2010-08-15	11
-292	val_292	2010-08-15	11
-219	val_219	2010-08-15	11
-287	val_287	2010-08-15	11
-153	val_153	2010-08-15	11
-193	val_193	2010-08-15	11
-338	val_338	2010-08-15	11
-446	val_446	2010-08-15	11
-459	val_459	2010-08-15	11
-394	val_394	2010-08-15	11
-237	val_237	2010-08-15	11
-482	val_482	2010-08-15	11
-174	val_174	2010-08-15	11
-413	val_413	2010-08-15	11
-494	val_494	2010-08-15	11
-207	val_207	2010-08-15	11
-199	val_199	2010-08-15	11
-466	val_466	2010-08-15	11
-208	val_208	2010-08-15	11
-174	val_174	2010-08-15	11
-399	val_399	2010-08-15	11
-396	val_396	2010-08-15	11
-247	val_247	2010-08-15	11
-417	val_417	2010-08-15	11
-489	val_489	2010-08-15	11
-162	val_162	2010-08-15	11
-377	val_377	2010-08-15	11
-397	val_397	2010-08-15	11
-309	val_309	2010-08-15	11
-365	val_365	2010-08-15	11
-266	val_266	2010-08-15	11
-439	val_439	2010-08-15	11
-342	val_342	2010-08-15	11
-367	val_367	2010-08-15	11
-325	val_325	2010-08-15	11
-167	val_167	2010-08-15	11
-195	val_195	2010-08-15	11
-475	val_475	2010-08-15	11
-17	val_17	2010-08-15	11
-113	val_113	2010-08-15	11
-155	val_155	2010-08-15	11
-203	val_203	2010-08-15	11
-339	val_339	2010-08-15	11
-0	val_0	2010-08-15	11
-455	val_455	2010-08-15	11
-128	val_128	2010-08-15	11
-311	val_311	2010-08-15	11
-316	val_316	2010-08-15	11
-57	val_57	2010-08-15	11
-302	val_302	2010-08-15	11
-205	val_205	2010-08-15	11
-149	val_149	2010-08-15	11
-438	val_438	2010-08-15	11
-345	val_345	2010-08-15	11
-129	val_129	2010-08-15	11
-170	val_170	2010-08-15	11
-20	val_20	2010-08-15	11
-489	val_489	2010-08-15	11
-157	val_157	2010-08-15	11
-378	val_378	2010-08-15	11
-221	val_221	2010-08-15	11
-92	val_92	2010-08-15	11
-111	val_111	2010-08-15	11
-47	val_47	2010-08-15	11
-72	val_72	2010-08-15	11
-4	val_4	2010-08-15	11
-280	val_280	2010-08-15	11
-35	val_35	2010-08-15	11
-427	val_427	2010-08-15	11
-277	val_277	2010-08-15	11
-208	val_208	2010-08-15	11
-356	val_356	2010-08-15	11
-399	val_399	2010-08-15	11
-169	val_169	2010-08-15	11
-382	val_382	2010-08-15	11
-498	val_498	2010-08-15	11
-125	val_125	2010-08-15	11
-386	val_386	2010-08-15	11
-437	val_437	2010-08-15	11
-469	val_469	2010-08-15	11
-192	val_192	2010-08-15	11
-286	val_286	2010-08-15	11
-187	val_187	2010-08-15	11
-176	val_176	2010-08-15	11
-54	val_54	2010-08-15	11
-459	val_459	2010-08-15	11
-51	val_51	2010-08-15	11
-138	val_138	2010-08-15	11
-103	val_103	2010-08-15	11
-239	val_239	2010-08-15	11
-213	val_213	2010-08-15	11
-216	val_216	2010-08-15	11
-430	val_430	2010-08-15	11
-278	val_278	2010-08-15	11
-176	val_176	2010-08-15	11
-289	val_289	2010-08-15	11
-221	val_221	2010-08-15	11
-65	val_65	2010-08-15	11
-318	val_318	2010-08-15	11
-332	val_332	2010-08-15	11
-311	val_311	2010-08-15	11
-275	val_275	2010-08-15	11
-137	val_137	2010-08-15	11
-241	val_241	2010-08-15	11
-83	val_83	2010-08-15	11
-333	val_333	2010-08-15	11
-180	val_180	2010-08-15	11
-284	val_284	2010-08-15	11
-12	val_12	2010-08-15	11
-230	val_230	2010-08-15	11
-181	val_181	2010-08-15	11
-67	val_67	2010-08-15	11
-260	val_260	2010-08-15	11
-404	val_404	2010-08-15	11
-384	val_384	2010-08-15	11
-489	val_489	2010-08-15	11
-353	val_353	2010-08-15	11
-373	val_373	2010-08-15	11
-272	val_272	2010-08-15	11
-138	val_138	2010-08-15	11
-217	val_217	2010-08-15	11
-84	val_84	2010-08-15	11
-348	val_348	2010-08-15	11
-466	val_466	2010-08-15	11
-58	val_58	2010-08-15	11
-8	val_8	2010-08-15	11
-411	val_411	2010-08-15	11
-230	val_230	2010-08-15	11
-208	val_208	2010-08-15	11
-348	val_348	2010-08-15	11
-24	val_24	2010-08-15	11
-463	val_463	2010-08-15	11
-431	val_431	2010-08-15	11
-179	val_179	2010-08-15	11
-172	val_172	2010-08-15	11
-42	val_42	2010-08-15	11
-129	val_129	2010-08-15	11
-158	val_158	2010-08-15	11
-119	val_119	2010-08-15	11
-496	val_496	2010-08-15	11
-0	val_0	2010-08-15	11
-322	val_322	2010-08-15	11
-197	val_197	2010-08-15	11
-468	val_468	2010-08-15	11
-393	val_393	2010-08-15	11
-454	val_454	2010-08-15	11
-100	val_100	2010-08-15	11
-298	val_298	2010-08-15	11
-199	val_199	2010-08-15	11
-191	val_191	2010-08-15	11
-418	val_418	2010-08-15	11
-96	val_96	2010-08-15	11
-26	val_26	2010-08-15	11
-165	val_165	2010-08-15	11
-327	val_327	2010-08-15	11
-230	val_230	2010-08-15	11
-205	val_205	2010-08-15	11
-120	val_120	2010-08-15	11
-131	val_131	2010-08-15	11
-51	val_51	2010-08-15	11
-404	val_404	2010-08-15	11
-43	val_43	2010-08-15	11
-436	val_436	2010-08-15	11
-156	val_156	2010-08-15	11
-469	val_469	2010-08-15	11
-468	val_468	2010-08-15	11
-308	val_308	2010-08-15	11
-95	val_95	2010-08-15	11
-196	val_196	2010-08-15	11
-288	val_288	2010-08-15	11
-481	val_481	2010-08-15	11
-457	val_457	2010-08-15	11
-98	val_98	2010-08-15	11
-282	val_282	2010-08-15	11
-197	val_197	2010-08-15	11
-187	val_187	2010-08-15	11
-318	val_318	2010-08-15	11
-318	val_318	2010-08-15	11
-409	val_409	2010-08-15	11
-470	val_470	2010-08-15	11
-137	val_137	2010-08-15	11
-369	val_369	2010-08-15	11
-316	val_316	2010-08-15	11
-169	val_169	2010-08-15	11
-413	val_413	2010-08-15	11
-85	val_85	2010-08-15	11
-77	val_77	2010-08-15	11
-0	val_0	2010-08-15	11
-490	val_490	2010-08-15	11
-87	val_87	2010-08-15	11
-364	val_364	2010-08-15	11
-179	val_179	2010-08-15	11
-118	val_118	2010-08-15	11
-134	val_134	2010-08-15	11
-395	val_395	2010-08-15	11
-282	val_282	2010-08-15	11
-138	val_138	2010-08-15	11
-238	val_238	2010-08-15	11
-419	val_419	2010-08-15	11
-15	val_15	2010-08-15	11
-118	val_118	2010-08-15	11
-72	val_72	2010-08-15	11
-90	val_90	2010-08-15	11
-307	val_307	2010-08-15	11
-19	val_19	2010-08-15	11
-435	val_435	2010-08-15	11
-10	val_10	2010-08-15	11
-277	val_277	2010-08-15	11
-273	val_273	2010-08-15	11
-306	val_306	2010-08-15	11
-224	val_224	2010-08-15	11
-309	val_309	2010-08-15	11
-389	val_389	2010-08-15	11
-327	val_327	2010-08-15	11
-242	val_242	2010-08-15	11
-369	val_369	2010-08-15	11
-392	val_392	2010-08-15	11
-272	val_272	2010-08-15	11
-331	val_331	2010-08-15	11
-401	val_401	2010-08-15	11
-242	val_242	2010-08-15	11
-452	val_452	2010-08-15	11
-177	val_177	2010-08-15	11
-226	val_226	2010-08-15	11
-5	val_5	2010-08-15	11
-497	val_497	2010-08-15	11
-402	val_402	2010-08-15	11
-396	val_396	2010-08-15	11
-317	val_317	2010-08-15	11
-395	val_395	2010-08-15	11
-58	val_58	2010-08-15	11
-35	val_35	2010-08-15	11
-336	val_336	2010-08-15	11
-95	val_95	2010-08-15	11
-11	val_11	2010-08-15	11
-168	val_168	2010-08-15	11
-34	val_34	2010-08-15	11
-229	val_229	2010-08-15	11
-233	val_233	2010-08-15	11
-143	val_143	2010-08-15	11
-472	val_472	2010-08-15	11
-322	val_322	2010-08-15	11
-498	val_498	2010-08-15	11
-160	val_160	2010-08-15	11
-195	val_195	2010-08-15	11
-42	val_42	2010-08-15	11
-321	val_321	2010-08-15	11
-430	val_430	2010-08-15	11
-119	val_119	2010-08-15	11
-489	val_489	2010-08-15	11
-458	val_458	2010-08-15	11
-78	val_78	2010-08-15	11
-76	val_76	2010-08-15	11
-41	val_41	2010-08-15	11
-223	val_223	2010-08-15	11
-492	val_492	2010-08-15	11
-149	val_149	2010-08-15	11
-449	val_449	2010-08-15	11
-218	val_218	2010-08-15	11
-228	val_228	2010-08-15	11
-138	val_138	2010-08-15	11
-453	val_453	2010-08-15	11
-30	val_30	2010-08-15	11
-209	val_209	2010-08-15	11
-64	val_64	2010-08-15	11
-468	val_468	2010-08-15	11
-76	val_76	2010-08-15	11
-74	val_74	2010-08-15	11
-342	val_342	2010-08-15	11
-69	val_69	2010-08-15	11
-230	val_230	2010-08-15	11
-33	val_33	2010-08-15	11
-368	val_368	2010-08-15	11
-103	val_103	2010-08-15	11
-296	val_296	2010-08-15	11
-113	val_113	2010-08-15	11
-216	val_216	2010-08-15	11
-367	val_367	2010-08-15	11
-344	val_344	2010-08-15	11
-167	val_167	2010-08-15	11
-274	val_274	2010-08-15	11
-219	val_219	2010-08-15	11
-239	val_239	2010-08-15	11
-485	val_485	2010-08-15	11
-116	val_116	2010-08-15	11
-223	val_223	2010-08-15	11
-256	val_256	2010-08-15	11
-263	val_263	2010-08-15	11
-70	val_70	2010-08-15	11
-487	val_487	2010-08-15	11
-480	val_480	2010-08-15	11
-401	val_401	2010-08-15	11
-288	val_288	2010-08-15	11
-191	val_191	2010-08-15	11
-5	val_5	2010-08-15	11
-244	val_244	2010-08-15	11
-438	val_438	2010-08-15	11
-128	val_128	2010-08-15	11
-467	val_467	2010-08-15	11
-432	val_432	2010-08-15	11
-202	val_202	2010-08-15	11
-316	val_316	2010-08-15	11
-229	val_229	2010-08-15	11
-469	val_469	2010-08-15	11
-463	val_463	2010-08-15	11
-280	val_280	2010-08-15	11
-2	val_2	2010-08-15	11
-35	val_35	2010-08-15	11
-283	val_283	2010-08-15	11
-331	val_331	2010-08-15	11
-235	val_235	2010-08-15	11
-80	val_80	2010-08-15	11
-44	val_44	2010-08-15	11
-193	val_193	2010-08-15	11
-321	val_321	2010-08-15	11
-335	val_335	2010-08-15	11
-104	val_104	2010-08-15	11
-466	val_466	2010-08-15	11
-366	val_366	2010-08-15	11
-175	val_175	2010-08-15	11
-403	val_403	2010-08-15	11
-483	val_483	2010-08-15	11
-53	val_53	2010-08-15	11
-105	val_105	2010-08-15	11
-257	val_257	2010-08-15	11
-406	val_406	2010-08-15	11
-409	val_409	2010-08-15	11
-190	val_190	2010-08-15	11
-406	val_406	2010-08-15	11
-401	val_401	2010-08-15	11
-114	val_114	2010-08-15	11
-258	val_258	2010-08-15	11
-90	val_90	2010-08-15	11
-203	val_203	2010-08-15	11
-262	val_262	2010-08-15	11
-348	val_348	2010-08-15	11
-424	val_424	2010-08-15	11
-12	val_12	2010-08-15	11
-396	val_396	2010-08-15	11
-201	val_201	2010-08-15	11
-217	val_217	2010-08-15	11
-164	val_164	2010-08-15	11
-431	val_431	2010-08-15	11
-454	val_454	2010-08-15	11
-478	val_478	2010-08-15	11
-298	val_298	2010-08-15	11
-125	val_125	2010-08-15	11
-431	val_431	2010-08-15	11
-164	val_164	2010-08-15	11
-424	val_424	2010-08-15	11
-187	val_187	2010-08-15	11
-382	val_382	2010-08-15	11
-5	val_5	2010-08-15	11
-70	val_70	2010-08-15	11
-397	val_397	2010-08-15	11
-480	val_480	2010-08-15	11
-291	val_291	2010-08-15	11
-24	val_24	2010-08-15	11
-351	val_351	2010-08-15	11
-255	val_255	2010-08-15	11
-104	val_104	2010-08-15	11
-70	val_70	2010-08-15	11
-163	val_163	2010-08-15	11
-438	val_438	2010-08-15	11
-119	val_119	2010-08-15	11
-414	val_414	2010-08-15	11
-200	val_200	2010-08-15	11
-491	val_491	2010-08-15	11
-237	val_237	2010-08-15	11
-439	val_439	2010-08-15	11
-360	val_360	2010-08-15	11
-248	val_248	2010-08-15	11
-479	val_479	2010-08-15	11
-305	val_305	2010-08-15	11
-417	val_417	2010-08-15	11
-199	val_199	2010-08-15	11
-444	val_444	2010-08-15	11
-120	val_120	2010-08-15	11
-429	val_429	2010-08-15	11
-169	val_169	2010-08-15	11
-443	val_443	2010-08-15	11
-323	val_323	2010-08-15	11
-325	val_325	2010-08-15	11
-277	val_277	2010-08-15	11
-230	val_230	2010-08-15	11
-478	val_478	2010-08-15	11
-178	val_178	2010-08-15	11
-468	val_468	2010-08-15	11
-310	val_310	2010-08-15	11
-317	val_317	2010-08-15	11
-333	val_333	2010-08-15	11
-493	val_493	2010-08-15	11
-460	val_460	2010-08-15	11
-207	val_207	2010-08-15	11
-249	val_249	2010-08-15	11
-265	val_265	2010-08-15	11
-480	val_480	2010-08-15	11
-83	val_83	2010-08-15	11
-136	val_136	2010-08-15	11
-353	val_353	2010-08-15	11
-172	val_172	2010-08-15	11
-214	val_214	2010-08-15	11
-462	val_462	2010-08-15	11
-233	val_233	2010-08-15	11
-406	val_406	2010-08-15	11
-133	val_133	2010-08-15	11
-175	val_175	2010-08-15	11
-189	val_189	2010-08-15	11
-454	val_454	2010-08-15	11
-375	val_375	2010-08-15	11
-401	val_401	2010-08-15	11
-421	val_421	2010-08-15	11
-407	val_407	2010-08-15	11
-384	val_384	2010-08-15	11
-256	val_256	2010-08-15	11
-26	val_26	2010-08-15	11
-134	val_134	2010-08-15	11
-67	val_67	2010-08-15	11
-384	val_384	2010-08-15	11
-379	val_379	2010-08-15	11
-18	val_18	2010-08-15	11
-462	val_462	2010-08-15	11
-492	val_492	2010-08-15	11
-100	val_100	2010-08-15	11
-298	val_298	2010-08-15	11
-9	val_9	2010-08-15	11
-341	val_341	2010-08-15	11
-498	val_498	2010-08-15	11
-146	val_146	2010-08-15	11
-458	val_458	2010-08-15	11
-362	val_362	2010-08-15	11
-186	val_186	2010-08-15	11
-285	val_285	2010-08-15	11
-348	val_348	2010-08-15	11
-167	val_167	2010-08-15	11
-18	val_18	2010-08-15	11
-273	val_273	2010-08-15	11
-183	val_183	2010-08-15	11
-281	val_281	2010-08-15	11
-344	val_344	2010-08-15	11
-97	val_97	2010-08-15	11
-469	val_469	2010-08-15	11
-315	val_315	2010-08-15	11
-84	val_84	2010-08-15	11
-28	val_28	2010-08-15	11
-37	val_37	2010-08-15	11
-448	val_448	2010-08-15	11
-152	val_152	2010-08-15	11
-348	val_348	2010-08-15	11
-307	val_307	2010-08-15	11
-194	val_194	2010-08-15	11
-414	val_414	2010-08-15	11
-477	val_477	2010-08-15	11
-222	val_222	2010-08-15	11
-126	val_126	2010-08-15	11
-90	val_90	2010-08-15	11
-169	val_169	2010-08-15	11
-403	val_403	2010-08-15	11
-400	val_400	2010-08-15	11
-200	val_200	2010-08-15	11
-97	val_97	2010-08-15	11
-238	val_238	2010-08-15	12
-86	val_86	2010-08-15	12
-311	val_311	2010-08-15	12
-27	val_27	2010-08-15	12
-165	val_165	2010-08-15	12
-409	val_409	2010-08-15	12
-255	val_255	2010-08-15	12
-278	val_278	2010-08-15	12
-98	val_98	2010-08-15	12
-484	val_484	2010-08-15	12
-265	val_265	2010-08-15	12
-193	val_193	2010-08-15	12
-401	val_401	2010-08-15	12
-150	val_150	2010-08-15	12
-273	val_273	2010-08-15	12
-224	val_224	2010-08-15	12
-369	val_369	2010-08-15	12
-66	val_66	2010-08-15	12
-128	val_128	2010-08-15	12
-213	val_213	2010-08-15	12
-146	val_146	2010-08-15	12
-406	val_406	2010-08-15	12
-429	val_429	2010-08-15	12
-374	val_374	2010-08-15	12
-152	val_152	2010-08-15	12
-469	val_469	2010-08-15	12
-145	val_145	2010-08-15	12
-495	val_495	2010-08-15	12
-37	val_37	2010-08-15	12
-327	val_327	2010-08-15	12
-281	val_281	2010-08-15	12
-277	val_277	2010-08-15	12
-209	val_209	2010-08-15	12
-15	val_15	2010-08-15	12
-82	val_82	2010-08-15	12
-403	val_403	2010-08-15	12
-166	val_166	2010-08-15	12
-417	val_417	2010-08-15	12
-430	val_430	2010-08-15	12
-252	val_252	2010-08-15	12
-292	val_292	2010-08-15	12
-219	val_219	2010-08-15	12
-287	val_287	2010-08-15	12
-153	val_153	2010-08-15	12
-193	val_193	2010-08-15	12
-338	val_338	2010-08-15	12
-446	val_446	2010-08-15	12
-459	val_459	2010-08-15	12
-394	val_394	2010-08-15	12
-237	val_237	2010-08-15	12
-482	val_482	2010-08-15	12
-174	val_174	2010-08-15	12
-413	val_413	2010-08-15	12
-494	val_494	2010-08-15	12
-207	val_207	2010-08-15	12
-199	val_199	2010-08-15	12
-466	val_466	2010-08-15	12
-208	val_208	2010-08-15	12
-174	val_174	2010-08-15	12
-399	val_399	2010-08-15	12
-396	val_396	2010-08-15	12
-247	val_247	2010-08-15	12
-417	val_417	2010-08-15	12
-489	val_489	2010-08-15	12
-162	val_162	2010-08-15	12
-377	val_377	2010-08-15	12
-397	val_397	2010-08-15	12
-309	val_309	2010-08-15	12
-365	val_365	2010-08-15	12
-266	val_266	2010-08-15	12
-439	val_439	2010-08-15	12
-342	val_342	2010-08-15	12
-367	val_367	2010-08-15	12
-325	val_325	2010-08-15	12
-167	val_167	2010-08-15	12
-195	val_195	2010-08-15	12
-475	val_475	2010-08-15	12
-17	val_17	2010-08-15	12
-113	val_113	2010-08-15	12
-155	val_155	2010-08-15	12
-203	val_203	2010-08-15	12
-339	val_339	2010-08-15	12
-0	val_0	2010-08-15	12
-455	val_455	2010-08-15	12
-128	val_128	2010-08-15	12
-311	val_311	2010-08-15	12
-316	val_316	2010-08-15	12
-57	val_57	2010-08-15	12
-302	val_302	2010-08-15	12
-205	val_205	2010-08-15	12
-149	val_149	2010-08-15	12
-438	val_438	2010-08-15	12
-345	val_345	2010-08-15	12
-129	val_129	2010-08-15	12
-170	val_170	2010-08-15	12
-20	val_20	2010-08-15	12
-489	val_489	2010-08-15	12
-157	val_157	2010-08-15	12
-378	val_378	2010-08-15	12
-221	val_221	2010-08-15	12
-92	val_92	2010-08-15	12
-111	val_111	2010-08-15	12
-47	val_47	2010-08-15	12
-72	val_72	2010-08-15	12
-4	val_4	2010-08-15	12
-280	val_280	2010-08-15	12
-35	val_35	2010-08-15	12
-427	val_427	2010-08-15	12
-277	val_277	2010-08-15	12
-208	val_208	2010-08-15	12
-356	val_356	2010-08-15	12
-399	val_399	2010-08-15	12
-169	val_169	2010-08-15	12
-382	val_382	2010-08-15	12
-498	val_498	2010-08-15	12
-125	val_125	2010-08-15	12
-386	val_386	2010-08-15	12
-437	val_437	2010-08-15	12
-469	val_469	2010-08-15	12
-192	val_192	2010-08-15	12
-286	val_286	2010-08-15	12
-187	val_187	2010-08-15	12
-176	val_176	2010-08-15	12
-54	val_54	2010-08-15	12
-459	val_459	2010-08-15	12
-51	val_51	2010-08-15	12
-138	val_138	2010-08-15	12
-103	val_103	2010-08-15	12
-239	val_239	2010-08-15	12
-213	val_213	2010-08-15	12
-216	val_216	2010-08-15	12
-430	val_430	2010-08-15	12
-278	val_278	2010-08-15	12
-176	val_176	2010-08-15	12
-289	val_289	2010-08-15	12
-221	val_221	2010-08-15	12
-65	val_65	2010-08-15	12
-318	val_318	2010-08-15	12
-332	val_332	2010-08-15	12
-311	val_311	2010-08-15	12
-275	val_275	2010-08-15	12
-137	val_137	2010-08-15	12
-241	val_241	2010-08-15	12
-83	val_83	2010-08-15	12
-333	val_333	2010-08-15	12
-180	val_180	2010-08-15	12
-284	val_284	2010-08-15	12
-12	val_12	2010-08-15	12
-230	val_230	2010-08-15	12
-181	val_181	2010-08-15	12
-67	val_67	2010-08-15	12
-260	val_260	2010-08-15	12
-404	val_404	2010-08-15	12
-384	val_384	2010-08-15	12
-489	val_489	2010-08-15	12
-353	val_353	2010-08-15	12
-373	val_373	2010-08-15	12
-272	val_272	2010-08-15	12
-138	val_138	2010-08-15	12
-217	val_217	2010-08-15	12
-84	val_84	2010-08-15	12
-348	val_348	2010-08-15	12
-466	val_466	2010-08-15	12
-58	val_58	2010-08-15	12
-8	val_8	2010-08-15	12
-411	val_411	2010-08-15	12
-230	val_230	2010-08-15	12
-208	val_208	2010-08-15	12
-348	val_348	2010-08-15	12
-24	val_24	2010-08-15	12
-463	val_463	2010-08-15	12
-431	val_431	2010-08-15	12
-179	val_179	2010-08-15	12
-172	val_172	2010-08-15	12
-42	val_42	2010-08-15	12
-129	val_129	2010-08-15	12
-158	val_158	2010-08-15	12
-119	val_119	2010-08-15	12
-496	val_496	2010-08-15	12
-0	val_0	2010-08-15	12
-322	val_322	2010-08-15	12
-197	val_197	2010-08-15	12
-468	val_468	2010-08-15	12
-393	val_393	2010-08-15	12
-454	val_454	2010-08-15	12
-100	val_100	2010-08-15	12
-298	val_298	2010-08-15	12
-199	val_199	2010-08-15	12
-191	val_191	2010-08-15	12
-418	val_418	2010-08-15	12
-96	val_96	2010-08-15	12
-26	val_26	2010-08-15	12
-165	val_165	2010-08-15	12
-327	val_327	2010-08-15	12
-230	val_230	2010-08-15	12
-205	val_205	2010-08-15	12
-120	val_120	2010-08-15	12
-131	val_131	2010-08-15	12
-51	val_51	2010-08-15	12
-404	val_404	2010-08-15	12
-43	val_43	2010-08-15	12
-436	val_436	2010-08-15	12
-156	val_156	2010-08-15	12
-469	val_469	2010-08-15	12
-468	val_468	2010-08-15	12
-308	val_308	2010-08-15	12
-95	val_95	2010-08-15	12
-196	val_196	2010-08-15	12
-288	val_288	2010-08-15	12
-481	val_481	2010-08-15	12
-457	val_457	2010-08-15	12
-98	val_98	2010-08-15	12
-282	val_282	2010-08-15	12
-197	val_197	2010-08-15	12
-187	val_187	2010-08-15	12
-318	val_318	2010-08-15	12
-318	val_318	2010-08-15	12
-409	val_409	2010-08-15	12
-470	val_470	2010-08-15	12
-137	val_137	2010-08-15	12
-369	val_369	2010-08-15	12
-316	val_316	2010-08-15	12
-169	val_169	2010-08-15	12
-413	val_413	2010-08-15	12
-85	val_85	2010-08-15	12
-77	val_77	2010-08-15	12
-0	val_0	2010-08-15	12
-490	val_490	2010-08-15	12
-87	val_87	2010-08-15	12
-364	val_364	2010-08-15	12
-179	val_179	2010-08-15	12
-118	val_118	2010-08-15	12
-134	val_134	2010-08-15	12
-395	val_395	2010-08-15	12
-282	val_282	2010-08-15	12
-138	val_138	2010-08-15	12
-238	val_238	2010-08-15	12
-419	val_419	2010-08-15	12
-15	val_15	2010-08-15	12
-118	val_118	2010-08-15	12
-72	val_72	2010-08-15	12
-90	val_90	2010-08-15	12
-307	val_307	2010-08-15	12
-19	val_19	2010-08-15	12
-435	val_435	2010-08-15	12
-10	val_10	2010-08-15	12
-277	val_277	2010-08-15	12
-273	val_273	2010-08-15	12
-306	val_306	2010-08-15	12
-224	val_224	2010-08-15	12
-309	val_309	2010-08-15	12
-389	val_389	2010-08-15	12
-327	val_327	2010-08-15	12
-242	val_242	2010-08-15	12
-369	val_369	2010-08-15	12
-392	val_392	2010-08-15	12
-272	val_272	2010-08-15	12
-331	val_331	2010-08-15	12
-401	val_401	2010-08-15	12
-242	val_242	2010-08-15	12
-452	val_452	2010-08-15	12
-177	val_177	2010-08-15	12
-226	val_226	2010-08-15	12
-5	val_5	2010-08-15	12
-497	val_497	2010-08-15	12
-402	val_402	2010-08-15	12
-396	val_396	2010-08-15	12
-317	val_317	2010-08-15	12
-395	val_395	2010-08-15	12
-58	val_58	2010-08-15	12
-35	val_35	2010-08-15	12
-336	val_336	2010-08-15	12
-95	val_95	2010-08-15	12
-11	val_11	2010-08-15	12
-168	val_168	2010-08-15	12
-34	val_34	2010-08-15	12
-229	val_229	2010-08-15	12
-233	val_233	2010-08-15	12
-143	val_143	2010-08-15	12
-472	val_472	2010-08-15	12
-322	val_322	2010-08-15	12
-498	val_498	2010-08-15	12
-160	val_160	2010-08-15	12
-195	val_195	2010-08-15	12
-42	val_42	2010-08-15	12
-321	val_321	2010-08-15	12
-430	val_430	2010-08-15	12
-119	val_119	2010-08-15	12
-489	val_489	2010-08-15	12
-458	val_458	2010-08-15	12
-78	val_78	2010-08-15	12
-76	val_76	2010-08-15	12
-41	val_41	2010-08-15	12
-223	val_223	2010-08-15	12
-492	val_492	2010-08-15	12
-149	val_149	2010-08-15	12
-449	val_449	2010-08-15	12
-218	val_218	2010-08-15	12
-228	val_228	2010-08-15	12
-138	val_138	2010-08-15	12
-453	val_453	2010-08-15	12
-30	val_30	2010-08-15	12
-209	val_209	2010-08-15	12
-64	val_64	2010-08-15	12
-468	val_468	2010-08-15	12
-76	val_76	2010-08-15	12
-74	val_74	2010-08-15	12
-342	val_342	2010-08-15	12
-69	val_69	2010-08-15	12
-230	val_230	2010-08-15	12
-33	val_33	2010-08-15	12
-368	val_368	2010-08-15	12
-103	val_103	2010-08-15	12
-296	val_296	2010-08-15	12
-113	val_113	2010-08-15	12
-216	val_216	2010-08-15	12
-367	val_367	2010-08-15	12
-344	val_344	2010-08-15	12
-167	val_167	2010-08-15	12
-274	val_274	2010-08-15	12
-219	val_219	2010-08-15	12
-239	val_239	2010-08-15	12
-485	val_485	2010-08-15	12
-116	val_116	2010-08-15	12
-223	val_223	2010-08-15	12
-256	val_256	2010-08-15	12
-263	val_263	2010-08-15	12
-70	val_70	2010-08-15	12
-487	val_487	2010-08-15	12
-480	val_480	2010-08-15	12
-401	val_401	2010-08-15	12
-288	val_288	2010-08-15	12
-191	val_191	2010-08-15	12
-5	val_5	2010-08-15	12
-244	val_244	2010-08-15	12
-438	val_438	2010-08-15	12
-128	val_128	2010-08-15	12
-467	val_467	2010-08-15	12
-432	val_432	2010-08-15	12
-202	val_202	2010-08-15	12
-316	val_316	2010-08-15	12
-229	val_229	2010-08-15	12
-469	val_469	2010-08-15	12
-463	val_463	2010-08-15	12
-280	val_280	2010-08-15	12
-2	val_2	2010-08-15	12
-35	val_35	2010-08-15	12
-283	val_283	2010-08-15	12
-331	val_331	2010-08-15	12
-235	val_235	2010-08-15	12
-80	val_80	2010-08-15	12
-44	val_44	2010-08-15	12
-193	val_193	2010-08-15	12
-321	val_321	2010-08-15	12
-335	val_335	2010-08-15	12
-104	val_104	2010-08-15	12
-466	val_466	2010-08-15	12
-366	val_366	2010-08-15	12
-175	val_175	2010-08-15	12
-403	val_403	2010-08-15	12
-483	val_483	2010-08-15	12
-53	val_53	2010-08-15	12
-105	val_105	2010-08-15	12
-257	val_257	2010-08-15	12
-406	val_406	2010-08-15	12
-409	val_409	2010-08-15	12
-190	val_190	2010-08-15	12
-406	val_406	2010-08-15	12
-401	val_401	2010-08-15	12
-114	val_114	2010-08-15	12
-258	val_258	2010-08-15	12
-90	val_90	2010-08-15	12
-203	val_203	2010-08-15	12
-262	val_262	2010-08-15	12
-348	val_348	2010-08-15	12
-424	val_424	2010-08-15	12
-12	val_12	2010-08-15	12
-396	val_396	2010-08-15	12
-201	val_201	2010-08-15	12
-217	val_217	2010-08-15	12
-164	val_164	2010-08-15	12
-431	val_431	2010-08-15	12
-454	val_454	2010-08-15	12
-478	val_478	2010-08-15	12
-298	val_298	2010-08-15	12
-125	val_125	2010-08-15	12
-431	val_431	2010-08-15	12
-164	val_164	2010-08-15	12
-424	val_424	2010-08-15	12
-187	val_187	2010-08-15	12
-382	val_382	2010-08-15	12
-5	val_5	2010-08-15	12
-70	val_70	2010-08-15	12
-397	val_397	2010-08-15	12
-480	val_480	2010-08-15	12
-291	val_291	2010-08-15	12
-24	val_24	2010-08-15	12
-351	val_351	2010-08-15	12
-255	val_255	2010-08-15	12
-104	val_104	2010-08-15	12
-70	val_70	2010-08-15	12
-163	val_163	2010-08-15	12
-438	val_438	2010-08-15	12
-119	val_119	2010-08-15	12
-414	val_414	2010-08-15	12
-200	val_200	2010-08-15	12
-491	val_491	2010-08-15	12
-237	val_237	2010-08-15	12
-439	val_439	2010-08-15	12
-360	val_360	2010-08-15	12
-248	val_248	2010-08-15	12
-479	val_479	2010-08-15	12
-305	val_305	2010-08-15	12
-417	val_417	2010-08-15	12
-199	val_199	2010-08-15	12
-444	val_444	2010-08-15	12
-120	val_120	2010-08-15	12
-429	val_429	2010-08-15	12
-169	val_169	2010-08-15	12
-443	val_443	2010-08-15	12
-323	val_323	2010-08-15	12
-325	val_325	2010-08-15	12
-277	val_277	2010-08-15	12
-230	val_230	2010-08-15	12
-478	val_478	2010-08-15	12
-178	val_178	2010-08-15	12
-468	val_468	2010-08-15	12
-310	val_310	2010-08-15	12
-317	val_317	2010-08-15	12
-333	val_333	2010-08-15	12
-493	val_493	2010-08-15	12
-460	val_460	2010-08-15	12
-207	val_207	2010-08-15	12
-249	val_249	2010-08-15	12
-265	val_265	2010-08-15	12
-480	val_480	2010-08-15	12
-83	val_83	2010-08-15	12
-136	val_136	2010-08-15	12
-353	val_353	2010-08-15	12
-172	val_172	2010-08-15	12
-214	val_214	2010-08-15	12
-462	val_462	2010-08-15	12
-233	val_233	2010-08-15	12
-406	val_406	2010-08-15	12
-133	val_133	2010-08-15	12
-175	val_175	2010-08-15	12
-189	val_189	2010-08-15	12
-454	val_454	2010-08-15	12
-375	val_375	2010-08-15	12
-401	val_401	2010-08-15	12
-421	val_421	2010-08-15	12
-407	val_407	2010-08-15	12
-384	val_384	2010-08-15	12
-256	val_256	2010-08-15	12
-26	val_26	2010-08-15	12
-134	val_134	2010-08-15	12
-67	val_67	2010-08-15	12
-384	val_384	2010-08-15	12
-379	val_379	2010-08-15	12
-18	val_18	2010-08-15	12
-462	val_462	2010-08-15	12
-492	val_492	2010-08-15	12
-100	val_100	2010-08-15	12
-298	val_298	2010-08-15	12
-9	val_9	2010-08-15	12
-341	val_341	2010-08-15	12
-498	val_498	2010-08-15	12
-146	val_146	2010-08-15	12
-458	val_458	2010-08-15	12
-362	val_362	2010-08-15	12
-186	val_186	2010-08-15	12
-285	val_285	2010-08-15	12
-348	val_348	2010-08-15	12
-167	val_167	2010-08-15	12
-18	val_18	2010-08-15	12
-273	val_273	2010-08-15	12
-183	val_183	2010-08-15	12
-281	val_281	2010-08-15	12
-344	val_344	2010-08-15	12
-97	val_97	2010-08-15	12
-469	val_469	2010-08-15	12
-315	val_315	2010-08-15	12
-84	val_84	2010-08-15	12
-28	val_28	2010-08-15	12
-37	val_37	2010-08-15	12
-448	val_448	2010-08-15	12
-152	val_152	2010-08-15	12
-348	val_348	2010-08-15	12
-307	val_307	2010-08-15	12
-194	val_194	2010-08-15	12
-414	val_414	2010-08-15	12
-477	val_477	2010-08-15	12
-222	val_222	2010-08-15	12
-126	val_126	2010-08-15	12
-90	val_90	2010-08-15	12
-169	val_169	2010-08-15	12
-403	val_403	2010-08-15	12
-400	val_400	2010-08-15	12
-200	val_200	2010-08-15	12
-97	val_97	2010-08-15	12
diff --git a/sql/hive/src/test/resources/golden/merge4-12-62541540a18d68a3cb8497a741061d11 b/sql/hive/src/test/resources/golden/merge4-12-62541540a18d68a3cb8497a741061d11
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 b/sql/hive/src/test/resources/golden/merge4-13-ed1103f06609365b40e78d13c654cc71
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 b/sql/hive/src/test/resources/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7
deleted file mode 100644
index 30becc42d7b5a2b5046cda62cd41bc656980b6d6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7
+++ /dev/null
@@ -1,3 +0,0 @@
-ds=2010-08-15/hr=11
-ds=2010-08-15/hr=12
-ds=2010-08-15/hr=file,
diff --git a/sql/hive/src/test/resources/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a b/sql/hive/src/test/resources/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a
deleted file mode 100644
index 4c867a5deff08262963305ec2dcb246354d6bf04..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a
+++ /dev/null
@@ -1 +0,0 @@
-1	1	2010-08-15	file,
diff --git a/sql/hive/src/test/resources/golden/merge4-5-3d24d877366c42030f6d9a596665720d b/sql/hive/src/test/resources/golden/merge4-5-3d24d877366c42030f6d9a596665720d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge4-6-b3a76420183795720ab3a384046e5af b/sql/hive/src/test/resources/golden/merge4-6-b3a76420183795720ab3a384046e5af
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d b/sql/hive/src/test/resources/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 b/sql/hive/src/test/resources/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868
deleted file mode 100644
index 9feba1dea5fd8a423c1d13dfc5b037bc6cb99f87..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868
+++ /dev/null
@@ -1,1000 +0,0 @@
-238	val_238	2010-08-15	11
-86	val_86	2010-08-15	11
-311	val_311	2010-08-15	11
-27	val_27	2010-08-15	11
-165	val_165	2010-08-15	11
-409	val_409	2010-08-15	11
-255	val_255	2010-08-15	11
-278	val_278	2010-08-15	11
-98	val_98	2010-08-15	11
-484	val_484	2010-08-15	11
-265	val_265	2010-08-15	11
-193	val_193	2010-08-15	11
-401	val_401	2010-08-15	11
-150	val_150	2010-08-15	11
-273	val_273	2010-08-15	11
-224	val_224	2010-08-15	11
-369	val_369	2010-08-15	11
-66	val_66	2010-08-15	11
-128	val_128	2010-08-15	11
-213	val_213	2010-08-15	11
-146	val_146	2010-08-15	11
-406	val_406	2010-08-15	11
-429	val_429	2010-08-15	11
-374	val_374	2010-08-15	11
-152	val_152	2010-08-15	11
-469	val_469	2010-08-15	11
-145	val_145	2010-08-15	11
-495	val_495	2010-08-15	11
-37	val_37	2010-08-15	11
-327	val_327	2010-08-15	11
-281	val_281	2010-08-15	11
-277	val_277	2010-08-15	11
-209	val_209	2010-08-15	11
-15	val_15	2010-08-15	11
-82	val_82	2010-08-15	11
-403	val_403	2010-08-15	11
-166	val_166	2010-08-15	11
-417	val_417	2010-08-15	11
-430	val_430	2010-08-15	11
-252	val_252	2010-08-15	11
-292	val_292	2010-08-15	11
-219	val_219	2010-08-15	11
-287	val_287	2010-08-15	11
-153	val_153	2010-08-15	11
-193	val_193	2010-08-15	11
-338	val_338	2010-08-15	11
-446	val_446	2010-08-15	11
-459	val_459	2010-08-15	11
-394	val_394	2010-08-15	11
-237	val_237	2010-08-15	11
-482	val_482	2010-08-15	11
-174	val_174	2010-08-15	11
-413	val_413	2010-08-15	11
-494	val_494	2010-08-15	11
-207	val_207	2010-08-15	11
-199	val_199	2010-08-15	11
-466	val_466	2010-08-15	11
-208	val_208	2010-08-15	11
-174	val_174	2010-08-15	11
-399	val_399	2010-08-15	11
-396	val_396	2010-08-15	11
-247	val_247	2010-08-15	11
-417	val_417	2010-08-15	11
-489	val_489	2010-08-15	11
-162	val_162	2010-08-15	11
-377	val_377	2010-08-15	11
-397	val_397	2010-08-15	11
-309	val_309	2010-08-15	11
-365	val_365	2010-08-15	11
-266	val_266	2010-08-15	11
-439	val_439	2010-08-15	11
-342	val_342	2010-08-15	11
-367	val_367	2010-08-15	11
-325	val_325	2010-08-15	11
-167	val_167	2010-08-15	11
-195	val_195	2010-08-15	11
-475	val_475	2010-08-15	11
-17	val_17	2010-08-15	11
-113	val_113	2010-08-15	11
-155	val_155	2010-08-15	11
-203	val_203	2010-08-15	11
-339	val_339	2010-08-15	11
-0	val_0	2010-08-15	11
-455	val_455	2010-08-15	11
-128	val_128	2010-08-15	11
-311	val_311	2010-08-15	11
-316	val_316	2010-08-15	11
-57	val_57	2010-08-15	11
-302	val_302	2010-08-15	11
-205	val_205	2010-08-15	11
-149	val_149	2010-08-15	11
-438	val_438	2010-08-15	11
-345	val_345	2010-08-15	11
-129	val_129	2010-08-15	11
-170	val_170	2010-08-15	11
-20	val_20	2010-08-15	11
-489	val_489	2010-08-15	11
-157	val_157	2010-08-15	11
-378	val_378	2010-08-15	11
-221	val_221	2010-08-15	11
-92	val_92	2010-08-15	11
-111	val_111	2010-08-15	11
-47	val_47	2010-08-15	11
-72	val_72	2010-08-15	11
-4	val_4	2010-08-15	11
-280	val_280	2010-08-15	11
-35	val_35	2010-08-15	11
-427	val_427	2010-08-15	11
-277	val_277	2010-08-15	11
-208	val_208	2010-08-15	11
-356	val_356	2010-08-15	11
-399	val_399	2010-08-15	11
-169	val_169	2010-08-15	11
-382	val_382	2010-08-15	11
-498	val_498	2010-08-15	11
-125	val_125	2010-08-15	11
-386	val_386	2010-08-15	11
-437	val_437	2010-08-15	11
-469	val_469	2010-08-15	11
-192	val_192	2010-08-15	11
-286	val_286	2010-08-15	11
-187	val_187	2010-08-15	11
-176	val_176	2010-08-15	11
-54	val_54	2010-08-15	11
-459	val_459	2010-08-15	11
-51	val_51	2010-08-15	11
-138	val_138	2010-08-15	11
-103	val_103	2010-08-15	11
-239	val_239	2010-08-15	11
-213	val_213	2010-08-15	11
-216	val_216	2010-08-15	11
-430	val_430	2010-08-15	11
-278	val_278	2010-08-15	11
-176	val_176	2010-08-15	11
-289	val_289	2010-08-15	11
-221	val_221	2010-08-15	11
-65	val_65	2010-08-15	11
-318	val_318	2010-08-15	11
-332	val_332	2010-08-15	11
-311	val_311	2010-08-15	11
-275	val_275	2010-08-15	11
-137	val_137	2010-08-15	11
-241	val_241	2010-08-15	11
-83	val_83	2010-08-15	11
-333	val_333	2010-08-15	11
-180	val_180	2010-08-15	11
-284	val_284	2010-08-15	11
-12	val_12	2010-08-15	11
-230	val_230	2010-08-15	11
-181	val_181	2010-08-15	11
-67	val_67	2010-08-15	11
-260	val_260	2010-08-15	11
-404	val_404	2010-08-15	11
-384	val_384	2010-08-15	11
-489	val_489	2010-08-15	11
-353	val_353	2010-08-15	11
-373	val_373	2010-08-15	11
-272	val_272	2010-08-15	11
-138	val_138	2010-08-15	11
-217	val_217	2010-08-15	11
-84	val_84	2010-08-15	11
-348	val_348	2010-08-15	11
-466	val_466	2010-08-15	11
-58	val_58	2010-08-15	11
-8	val_8	2010-08-15	11
-411	val_411	2010-08-15	11
-230	val_230	2010-08-15	11
-208	val_208	2010-08-15	11
-348	val_348	2010-08-15	11
-24	val_24	2010-08-15	11
-463	val_463	2010-08-15	11
-431	val_431	2010-08-15	11
-179	val_179	2010-08-15	11
-172	val_172	2010-08-15	11
-42	val_42	2010-08-15	11
-129	val_129	2010-08-15	11
-158	val_158	2010-08-15	11
-119	val_119	2010-08-15	11
-496	val_496	2010-08-15	11
-0	val_0	2010-08-15	11
-322	val_322	2010-08-15	11
-197	val_197	2010-08-15	11
-468	val_468	2010-08-15	11
-393	val_393	2010-08-15	11
-454	val_454	2010-08-15	11
-100	val_100	2010-08-15	11
-298	val_298	2010-08-15	11
-199	val_199	2010-08-15	11
-191	val_191	2010-08-15	11
-418	val_418	2010-08-15	11
-96	val_96	2010-08-15	11
-26	val_26	2010-08-15	11
-165	val_165	2010-08-15	11
-327	val_327	2010-08-15	11
-230	val_230	2010-08-15	11
-205	val_205	2010-08-15	11
-120	val_120	2010-08-15	11
-131	val_131	2010-08-15	11
-51	val_51	2010-08-15	11
-404	val_404	2010-08-15	11
-43	val_43	2010-08-15	11
-436	val_436	2010-08-15	11
-156	val_156	2010-08-15	11
-469	val_469	2010-08-15	11
-468	val_468	2010-08-15	11
-308	val_308	2010-08-15	11
-95	val_95	2010-08-15	11
-196	val_196	2010-08-15	11
-288	val_288	2010-08-15	11
-481	val_481	2010-08-15	11
-457	val_457	2010-08-15	11
-98	val_98	2010-08-15	11
-282	val_282	2010-08-15	11
-197	val_197	2010-08-15	11
-187	val_187	2010-08-15	11
-318	val_318	2010-08-15	11
-318	val_318	2010-08-15	11
-409	val_409	2010-08-15	11
-470	val_470	2010-08-15	11
-137	val_137	2010-08-15	11
-369	val_369	2010-08-15	11
-316	val_316	2010-08-15	11
-169	val_169	2010-08-15	11
-413	val_413	2010-08-15	11
-85	val_85	2010-08-15	11
-77	val_77	2010-08-15	11
-0	val_0	2010-08-15	11
-490	val_490	2010-08-15	11
-87	val_87	2010-08-15	11
-364	val_364	2010-08-15	11
-179	val_179	2010-08-15	11
-118	val_118	2010-08-15	11
-134	val_134	2010-08-15	11
-395	val_395	2010-08-15	11
-282	val_282	2010-08-15	11
-138	val_138	2010-08-15	11
-238	val_238	2010-08-15	11
-419	val_419	2010-08-15	11
-15	val_15	2010-08-15	11
-118	val_118	2010-08-15	11
-72	val_72	2010-08-15	11
-90	val_90	2010-08-15	11
-307	val_307	2010-08-15	11
-19	val_19	2010-08-15	11
-435	val_435	2010-08-15	11
-10	val_10	2010-08-15	11
-277	val_277	2010-08-15	11
-273	val_273	2010-08-15	11
-306	val_306	2010-08-15	11
-224	val_224	2010-08-15	11
-309	val_309	2010-08-15	11
-389	val_389	2010-08-15	11
-327	val_327	2010-08-15	11
-242	val_242	2010-08-15	11
-369	val_369	2010-08-15	11
-392	val_392	2010-08-15	11
-272	val_272	2010-08-15	11
-331	val_331	2010-08-15	11
-401	val_401	2010-08-15	11
-242	val_242	2010-08-15	11
-452	val_452	2010-08-15	11
-177	val_177	2010-08-15	11
-226	val_226	2010-08-15	11
-5	val_5	2010-08-15	11
-497	val_497	2010-08-15	11
-402	val_402	2010-08-15	11
-396	val_396	2010-08-15	11
-317	val_317	2010-08-15	11
-395	val_395	2010-08-15	11
-58	val_58	2010-08-15	11
-35	val_35	2010-08-15	11
-336	val_336	2010-08-15	11
-95	val_95	2010-08-15	11
-11	val_11	2010-08-15	11
-168	val_168	2010-08-15	11
-34	val_34	2010-08-15	11
-229	val_229	2010-08-15	11
-233	val_233	2010-08-15	11
-143	val_143	2010-08-15	11
-472	val_472	2010-08-15	11
-322	val_322	2010-08-15	11
-498	val_498	2010-08-15	11
-160	val_160	2010-08-15	11
-195	val_195	2010-08-15	11
-42	val_42	2010-08-15	11
-321	val_321	2010-08-15	11
-430	val_430	2010-08-15	11
-119	val_119	2010-08-15	11
-489	val_489	2010-08-15	11
-458	val_458	2010-08-15	11
-78	val_78	2010-08-15	11
-76	val_76	2010-08-15	11
-41	val_41	2010-08-15	11
-223	val_223	2010-08-15	11
-492	val_492	2010-08-15	11
-149	val_149	2010-08-15	11
-449	val_449	2010-08-15	11
-218	val_218	2010-08-15	11
-228	val_228	2010-08-15	11
-138	val_138	2010-08-15	11
-453	val_453	2010-08-15	11
-30	val_30	2010-08-15	11
-209	val_209	2010-08-15	11
-64	val_64	2010-08-15	11
-468	val_468	2010-08-15	11
-76	val_76	2010-08-15	11
-74	val_74	2010-08-15	11
-342	val_342	2010-08-15	11
-69	val_69	2010-08-15	11
-230	val_230	2010-08-15	11
-33	val_33	2010-08-15	11
-368	val_368	2010-08-15	11
-103	val_103	2010-08-15	11
-296	val_296	2010-08-15	11
-113	val_113	2010-08-15	11
-216	val_216	2010-08-15	11
-367	val_367	2010-08-15	11
-344	val_344	2010-08-15	11
-167	val_167	2010-08-15	11
-274	val_274	2010-08-15	11
-219	val_219	2010-08-15	11
-239	val_239	2010-08-15	11
-485	val_485	2010-08-15	11
-116	val_116	2010-08-15	11
-223	val_223	2010-08-15	11
-256	val_256	2010-08-15	11
-263	val_263	2010-08-15	11
-70	val_70	2010-08-15	11
-487	val_487	2010-08-15	11
-480	val_480	2010-08-15	11
-401	val_401	2010-08-15	11
-288	val_288	2010-08-15	11
-191	val_191	2010-08-15	11
-5	val_5	2010-08-15	11
-244	val_244	2010-08-15	11
-438	val_438	2010-08-15	11
-128	val_128	2010-08-15	11
-467	val_467	2010-08-15	11
-432	val_432	2010-08-15	11
-202	val_202	2010-08-15	11
-316	val_316	2010-08-15	11
-229	val_229	2010-08-15	11
-469	val_469	2010-08-15	11
-463	val_463	2010-08-15	11
-280	val_280	2010-08-15	11
-2	val_2	2010-08-15	11
-35	val_35	2010-08-15	11
-283	val_283	2010-08-15	11
-331	val_331	2010-08-15	11
-235	val_235	2010-08-15	11
-80	val_80	2010-08-15	11
-44	val_44	2010-08-15	11
-193	val_193	2010-08-15	11
-321	val_321	2010-08-15	11
-335	val_335	2010-08-15	11
-104	val_104	2010-08-15	11
-466	val_466	2010-08-15	11
-366	val_366	2010-08-15	11
-175	val_175	2010-08-15	11
-403	val_403	2010-08-15	11
-483	val_483	2010-08-15	11
-53	val_53	2010-08-15	11
-105	val_105	2010-08-15	11
-257	val_257	2010-08-15	11
-406	val_406	2010-08-15	11
-409	val_409	2010-08-15	11
-190	val_190	2010-08-15	11
-406	val_406	2010-08-15	11
-401	val_401	2010-08-15	11
-114	val_114	2010-08-15	11
-258	val_258	2010-08-15	11
-90	val_90	2010-08-15	11
-203	val_203	2010-08-15	11
-262	val_262	2010-08-15	11
-348	val_348	2010-08-15	11
-424	val_424	2010-08-15	11
-12	val_12	2010-08-15	11
-396	val_396	2010-08-15	11
-201	val_201	2010-08-15	11
-217	val_217	2010-08-15	11
-164	val_164	2010-08-15	11
-431	val_431	2010-08-15	11
-454	val_454	2010-08-15	11
-478	val_478	2010-08-15	11
-298	val_298	2010-08-15	11
-125	val_125	2010-08-15	11
-431	val_431	2010-08-15	11
-164	val_164	2010-08-15	11
-424	val_424	2010-08-15	11
-187	val_187	2010-08-15	11
-382	val_382	2010-08-15	11
-5	val_5	2010-08-15	11
-70	val_70	2010-08-15	11
-397	val_397	2010-08-15	11
-480	val_480	2010-08-15	11
-291	val_291	2010-08-15	11
-24	val_24	2010-08-15	11
-351	val_351	2010-08-15	11
-255	val_255	2010-08-15	11
-104	val_104	2010-08-15	11
-70	val_70	2010-08-15	11
-163	val_163	2010-08-15	11
-438	val_438	2010-08-15	11
-119	val_119	2010-08-15	11
-414	val_414	2010-08-15	11
-200	val_200	2010-08-15	11
-491	val_491	2010-08-15	11
-237	val_237	2010-08-15	11
-439	val_439	2010-08-15	11
-360	val_360	2010-08-15	11
-248	val_248	2010-08-15	11
-479	val_479	2010-08-15	11
-305	val_305	2010-08-15	11
-417	val_417	2010-08-15	11
-199	val_199	2010-08-15	11
-444	val_444	2010-08-15	11
-120	val_120	2010-08-15	11
-429	val_429	2010-08-15	11
-169	val_169	2010-08-15	11
-443	val_443	2010-08-15	11
-323	val_323	2010-08-15	11
-325	val_325	2010-08-15	11
-277	val_277	2010-08-15	11
-230	val_230	2010-08-15	11
-478	val_478	2010-08-15	11
-178	val_178	2010-08-15	11
-468	val_468	2010-08-15	11
-310	val_310	2010-08-15	11
-317	val_317	2010-08-15	11
-333	val_333	2010-08-15	11
-493	val_493	2010-08-15	11
-460	val_460	2010-08-15	11
-207	val_207	2010-08-15	11
-249	val_249	2010-08-15	11
-265	val_265	2010-08-15	11
-480	val_480	2010-08-15	11
-83	val_83	2010-08-15	11
-136	val_136	2010-08-15	11
-353	val_353	2010-08-15	11
-172	val_172	2010-08-15	11
-214	val_214	2010-08-15	11
-462	val_462	2010-08-15	11
-233	val_233	2010-08-15	11
-406	val_406	2010-08-15	11
-133	val_133	2010-08-15	11
-175	val_175	2010-08-15	11
-189	val_189	2010-08-15	11
-454	val_454	2010-08-15	11
-375	val_375	2010-08-15	11
-401	val_401	2010-08-15	11
-421	val_421	2010-08-15	11
-407	val_407	2010-08-15	11
-384	val_384	2010-08-15	11
-256	val_256	2010-08-15	11
-26	val_26	2010-08-15	11
-134	val_134	2010-08-15	11
-67	val_67	2010-08-15	11
-384	val_384	2010-08-15	11
-379	val_379	2010-08-15	11
-18	val_18	2010-08-15	11
-462	val_462	2010-08-15	11
-492	val_492	2010-08-15	11
-100	val_100	2010-08-15	11
-298	val_298	2010-08-15	11
-9	val_9	2010-08-15	11
-341	val_341	2010-08-15	11
-498	val_498	2010-08-15	11
-146	val_146	2010-08-15	11
-458	val_458	2010-08-15	11
-362	val_362	2010-08-15	11
-186	val_186	2010-08-15	11
-285	val_285	2010-08-15	11
-348	val_348	2010-08-15	11
-167	val_167	2010-08-15	11
-18	val_18	2010-08-15	11
-273	val_273	2010-08-15	11
-183	val_183	2010-08-15	11
-281	val_281	2010-08-15	11
-344	val_344	2010-08-15	11
-97	val_97	2010-08-15	11
-469	val_469	2010-08-15	11
-315	val_315	2010-08-15	11
-84	val_84	2010-08-15	11
-28	val_28	2010-08-15	11
-37	val_37	2010-08-15	11
-448	val_448	2010-08-15	11
-152	val_152	2010-08-15	11
-348	val_348	2010-08-15	11
-307	val_307	2010-08-15	11
-194	val_194	2010-08-15	11
-414	val_414	2010-08-15	11
-477	val_477	2010-08-15	11
-222	val_222	2010-08-15	11
-126	val_126	2010-08-15	11
-90	val_90	2010-08-15	11
-169	val_169	2010-08-15	11
-403	val_403	2010-08-15	11
-400	val_400	2010-08-15	11
-200	val_200	2010-08-15	11
-97	val_97	2010-08-15	11
-238	val_238	2010-08-15	12
-86	val_86	2010-08-15	12
-311	val_311	2010-08-15	12
-27	val_27	2010-08-15	12
-165	val_165	2010-08-15	12
-409	val_409	2010-08-15	12
-255	val_255	2010-08-15	12
-278	val_278	2010-08-15	12
-98	val_98	2010-08-15	12
-484	val_484	2010-08-15	12
-265	val_265	2010-08-15	12
-193	val_193	2010-08-15	12
-401	val_401	2010-08-15	12
-150	val_150	2010-08-15	12
-273	val_273	2010-08-15	12
-224	val_224	2010-08-15	12
-369	val_369	2010-08-15	12
-66	val_66	2010-08-15	12
-128	val_128	2010-08-15	12
-213	val_213	2010-08-15	12
-146	val_146	2010-08-15	12
-406	val_406	2010-08-15	12
-429	val_429	2010-08-15	12
-374	val_374	2010-08-15	12
-152	val_152	2010-08-15	12
-469	val_469	2010-08-15	12
-145	val_145	2010-08-15	12
-495	val_495	2010-08-15	12
-37	val_37	2010-08-15	12
-327	val_327	2010-08-15	12
-281	val_281	2010-08-15	12
-277	val_277	2010-08-15	12
-209	val_209	2010-08-15	12
-15	val_15	2010-08-15	12
-82	val_82	2010-08-15	12
-403	val_403	2010-08-15	12
-166	val_166	2010-08-15	12
-417	val_417	2010-08-15	12
-430	val_430	2010-08-15	12
-252	val_252	2010-08-15	12
-292	val_292	2010-08-15	12
-219	val_219	2010-08-15	12
-287	val_287	2010-08-15	12
-153	val_153	2010-08-15	12
-193	val_193	2010-08-15	12
-338	val_338	2010-08-15	12
-446	val_446	2010-08-15	12
-459	val_459	2010-08-15	12
-394	val_394	2010-08-15	12
-237	val_237	2010-08-15	12
-482	val_482	2010-08-15	12
-174	val_174	2010-08-15	12
-413	val_413	2010-08-15	12
-494	val_494	2010-08-15	12
-207	val_207	2010-08-15	12
-199	val_199	2010-08-15	12
-466	val_466	2010-08-15	12
-208	val_208	2010-08-15	12
-174	val_174	2010-08-15	12
-399	val_399	2010-08-15	12
-396	val_396	2010-08-15	12
-247	val_247	2010-08-15	12
-417	val_417	2010-08-15	12
-489	val_489	2010-08-15	12
-162	val_162	2010-08-15	12
-377	val_377	2010-08-15	12
-397	val_397	2010-08-15	12
-309	val_309	2010-08-15	12
-365	val_365	2010-08-15	12
-266	val_266	2010-08-15	12
-439	val_439	2010-08-15	12
-342	val_342	2010-08-15	12
-367	val_367	2010-08-15	12
-325	val_325	2010-08-15	12
-167	val_167	2010-08-15	12
-195	val_195	2010-08-15	12
-475	val_475	2010-08-15	12
-17	val_17	2010-08-15	12
-113	val_113	2010-08-15	12
-155	val_155	2010-08-15	12
-203	val_203	2010-08-15	12
-339	val_339	2010-08-15	12
-0	val_0	2010-08-15	12
-455	val_455	2010-08-15	12
-128	val_128	2010-08-15	12
-311	val_311	2010-08-15	12
-316	val_316	2010-08-15	12
-57	val_57	2010-08-15	12
-302	val_302	2010-08-15	12
-205	val_205	2010-08-15	12
-149	val_149	2010-08-15	12
-438	val_438	2010-08-15	12
-345	val_345	2010-08-15	12
-129	val_129	2010-08-15	12
-170	val_170	2010-08-15	12
-20	val_20	2010-08-15	12
-489	val_489	2010-08-15	12
-157	val_157	2010-08-15	12
-378	val_378	2010-08-15	12
-221	val_221	2010-08-15	12
-92	val_92	2010-08-15	12
-111	val_111	2010-08-15	12
-47	val_47	2010-08-15	12
-72	val_72	2010-08-15	12
-4	val_4	2010-08-15	12
-280	val_280	2010-08-15	12
-35	val_35	2010-08-15	12
-427	val_427	2010-08-15	12
-277	val_277	2010-08-15	12
-208	val_208	2010-08-15	12
-356	val_356	2010-08-15	12
-399	val_399	2010-08-15	12
-169	val_169	2010-08-15	12
-382	val_382	2010-08-15	12
-498	val_498	2010-08-15	12
-125	val_125	2010-08-15	12
-386	val_386	2010-08-15	12
-437	val_437	2010-08-15	12
-469	val_469	2010-08-15	12
-192	val_192	2010-08-15	12
-286	val_286	2010-08-15	12
-187	val_187	2010-08-15	12
-176	val_176	2010-08-15	12
-54	val_54	2010-08-15	12
-459	val_459	2010-08-15	12
-51	val_51	2010-08-15	12
-138	val_138	2010-08-15	12
-103	val_103	2010-08-15	12
-239	val_239	2010-08-15	12
-213	val_213	2010-08-15	12
-216	val_216	2010-08-15	12
-430	val_430	2010-08-15	12
-278	val_278	2010-08-15	12
-176	val_176	2010-08-15	12
-289	val_289	2010-08-15	12
-221	val_221	2010-08-15	12
-65	val_65	2010-08-15	12
-318	val_318	2010-08-15	12
-332	val_332	2010-08-15	12
-311	val_311	2010-08-15	12
-275	val_275	2010-08-15	12
-137	val_137	2010-08-15	12
-241	val_241	2010-08-15	12
-83	val_83	2010-08-15	12
-333	val_333	2010-08-15	12
-180	val_180	2010-08-15	12
-284	val_284	2010-08-15	12
-12	val_12	2010-08-15	12
-230	val_230	2010-08-15	12
-181	val_181	2010-08-15	12
-67	val_67	2010-08-15	12
-260	val_260	2010-08-15	12
-404	val_404	2010-08-15	12
-384	val_384	2010-08-15	12
-489	val_489	2010-08-15	12
-353	val_353	2010-08-15	12
-373	val_373	2010-08-15	12
-272	val_272	2010-08-15	12
-138	val_138	2010-08-15	12
-217	val_217	2010-08-15	12
-84	val_84	2010-08-15	12
-348	val_348	2010-08-15	12
-466	val_466	2010-08-15	12
-58	val_58	2010-08-15	12
-8	val_8	2010-08-15	12
-411	val_411	2010-08-15	12
-230	val_230	2010-08-15	12
-208	val_208	2010-08-15	12
-348	val_348	2010-08-15	12
-24	val_24	2010-08-15	12
-463	val_463	2010-08-15	12
-431	val_431	2010-08-15	12
-179	val_179	2010-08-15	12
-172	val_172	2010-08-15	12
-42	val_42	2010-08-15	12
-129	val_129	2010-08-15	12
-158	val_158	2010-08-15	12
-119	val_119	2010-08-15	12
-496	val_496	2010-08-15	12
-0	val_0	2010-08-15	12
-322	val_322	2010-08-15	12
-197	val_197	2010-08-15	12
-468	val_468	2010-08-15	12
-393	val_393	2010-08-15	12
-454	val_454	2010-08-15	12
-100	val_100	2010-08-15	12
-298	val_298	2010-08-15	12
-199	val_199	2010-08-15	12
-191	val_191	2010-08-15	12
-418	val_418	2010-08-15	12
-96	val_96	2010-08-15	12
-26	val_26	2010-08-15	12
-165	val_165	2010-08-15	12
-327	val_327	2010-08-15	12
-230	val_230	2010-08-15	12
-205	val_205	2010-08-15	12
-120	val_120	2010-08-15	12
-131	val_131	2010-08-15	12
-51	val_51	2010-08-15	12
-404	val_404	2010-08-15	12
-43	val_43	2010-08-15	12
-436	val_436	2010-08-15	12
-156	val_156	2010-08-15	12
-469	val_469	2010-08-15	12
-468	val_468	2010-08-15	12
-308	val_308	2010-08-15	12
-95	val_95	2010-08-15	12
-196	val_196	2010-08-15	12
-288	val_288	2010-08-15	12
-481	val_481	2010-08-15	12
-457	val_457	2010-08-15	12
-98	val_98	2010-08-15	12
-282	val_282	2010-08-15	12
-197	val_197	2010-08-15	12
-187	val_187	2010-08-15	12
-318	val_318	2010-08-15	12
-318	val_318	2010-08-15	12
-409	val_409	2010-08-15	12
-470	val_470	2010-08-15	12
-137	val_137	2010-08-15	12
-369	val_369	2010-08-15	12
-316	val_316	2010-08-15	12
-169	val_169	2010-08-15	12
-413	val_413	2010-08-15	12
-85	val_85	2010-08-15	12
-77	val_77	2010-08-15	12
-0	val_0	2010-08-15	12
-490	val_490	2010-08-15	12
-87	val_87	2010-08-15	12
-364	val_364	2010-08-15	12
-179	val_179	2010-08-15	12
-118	val_118	2010-08-15	12
-134	val_134	2010-08-15	12
-395	val_395	2010-08-15	12
-282	val_282	2010-08-15	12
-138	val_138	2010-08-15	12
-238	val_238	2010-08-15	12
-419	val_419	2010-08-15	12
-15	val_15	2010-08-15	12
-118	val_118	2010-08-15	12
-72	val_72	2010-08-15	12
-90	val_90	2010-08-15	12
-307	val_307	2010-08-15	12
-19	val_19	2010-08-15	12
-435	val_435	2010-08-15	12
-10	val_10	2010-08-15	12
-277	val_277	2010-08-15	12
-273	val_273	2010-08-15	12
-306	val_306	2010-08-15	12
-224	val_224	2010-08-15	12
-309	val_309	2010-08-15	12
-389	val_389	2010-08-15	12
-327	val_327	2010-08-15	12
-242	val_242	2010-08-15	12
-369	val_369	2010-08-15	12
-392	val_392	2010-08-15	12
-272	val_272	2010-08-15	12
-331	val_331	2010-08-15	12
-401	val_401	2010-08-15	12
-242	val_242	2010-08-15	12
-452	val_452	2010-08-15	12
-177	val_177	2010-08-15	12
-226	val_226	2010-08-15	12
-5	val_5	2010-08-15	12
-497	val_497	2010-08-15	12
-402	val_402	2010-08-15	12
-396	val_396	2010-08-15	12
-317	val_317	2010-08-15	12
-395	val_395	2010-08-15	12
-58	val_58	2010-08-15	12
-35	val_35	2010-08-15	12
-336	val_336	2010-08-15	12
-95	val_95	2010-08-15	12
-11	val_11	2010-08-15	12
-168	val_168	2010-08-15	12
-34	val_34	2010-08-15	12
-229	val_229	2010-08-15	12
-233	val_233	2010-08-15	12
-143	val_143	2010-08-15	12
-472	val_472	2010-08-15	12
-322	val_322	2010-08-15	12
-498	val_498	2010-08-15	12
-160	val_160	2010-08-15	12
-195	val_195	2010-08-15	12
-42	val_42	2010-08-15	12
-321	val_321	2010-08-15	12
-430	val_430	2010-08-15	12
-119	val_119	2010-08-15	12
-489	val_489	2010-08-15	12
-458	val_458	2010-08-15	12
-78	val_78	2010-08-15	12
-76	val_76	2010-08-15	12
-41	val_41	2010-08-15	12
-223	val_223	2010-08-15	12
-492	val_492	2010-08-15	12
-149	val_149	2010-08-15	12
-449	val_449	2010-08-15	12
-218	val_218	2010-08-15	12
-228	val_228	2010-08-15	12
-138	val_138	2010-08-15	12
-453	val_453	2010-08-15	12
-30	val_30	2010-08-15	12
-209	val_209	2010-08-15	12
-64	val_64	2010-08-15	12
-468	val_468	2010-08-15	12
-76	val_76	2010-08-15	12
-74	val_74	2010-08-15	12
-342	val_342	2010-08-15	12
-69	val_69	2010-08-15	12
-230	val_230	2010-08-15	12
-33	val_33	2010-08-15	12
-368	val_368	2010-08-15	12
-103	val_103	2010-08-15	12
-296	val_296	2010-08-15	12
-113	val_113	2010-08-15	12
-216	val_216	2010-08-15	12
-367	val_367	2010-08-15	12
-344	val_344	2010-08-15	12
-167	val_167	2010-08-15	12
-274	val_274	2010-08-15	12
-219	val_219	2010-08-15	12
-239	val_239	2010-08-15	12
-485	val_485	2010-08-15	12
-116	val_116	2010-08-15	12
-223	val_223	2010-08-15	12
-256	val_256	2010-08-15	12
-263	val_263	2010-08-15	12
-70	val_70	2010-08-15	12
-487	val_487	2010-08-15	12
-480	val_480	2010-08-15	12
-401	val_401	2010-08-15	12
-288	val_288	2010-08-15	12
-191	val_191	2010-08-15	12
-5	val_5	2010-08-15	12
-244	val_244	2010-08-15	12
-438	val_438	2010-08-15	12
-128	val_128	2010-08-15	12
-467	val_467	2010-08-15	12
-432	val_432	2010-08-15	12
-202	val_202	2010-08-15	12
-316	val_316	2010-08-15	12
-229	val_229	2010-08-15	12
-469	val_469	2010-08-15	12
-463	val_463	2010-08-15	12
-280	val_280	2010-08-15	12
-2	val_2	2010-08-15	12
-35	val_35	2010-08-15	12
-283	val_283	2010-08-15	12
-331	val_331	2010-08-15	12
-235	val_235	2010-08-15	12
-80	val_80	2010-08-15	12
-44	val_44	2010-08-15	12
-193	val_193	2010-08-15	12
-321	val_321	2010-08-15	12
-335	val_335	2010-08-15	12
-104	val_104	2010-08-15	12
-466	val_466	2010-08-15	12
-366	val_366	2010-08-15	12
-175	val_175	2010-08-15	12
-403	val_403	2010-08-15	12
-483	val_483	2010-08-15	12
-53	val_53	2010-08-15	12
-105	val_105	2010-08-15	12
-257	val_257	2010-08-15	12
-406	val_406	2010-08-15	12
-409	val_409	2010-08-15	12
-190	val_190	2010-08-15	12
-406	val_406	2010-08-15	12
-401	val_401	2010-08-15	12
-114	val_114	2010-08-15	12
-258	val_258	2010-08-15	12
-90	val_90	2010-08-15	12
-203	val_203	2010-08-15	12
-262	val_262	2010-08-15	12
-348	val_348	2010-08-15	12
-424	val_424	2010-08-15	12
-12	val_12	2010-08-15	12
-396	val_396	2010-08-15	12
-201	val_201	2010-08-15	12
-217	val_217	2010-08-15	12
-164	val_164	2010-08-15	12
-431	val_431	2010-08-15	12
-454	val_454	2010-08-15	12
-478	val_478	2010-08-15	12
-298	val_298	2010-08-15	12
-125	val_125	2010-08-15	12
-431	val_431	2010-08-15	12
-164	val_164	2010-08-15	12
-424	val_424	2010-08-15	12
-187	val_187	2010-08-15	12
-382	val_382	2010-08-15	12
-5	val_5	2010-08-15	12
-70	val_70	2010-08-15	12
-397	val_397	2010-08-15	12
-480	val_480	2010-08-15	12
-291	val_291	2010-08-15	12
-24	val_24	2010-08-15	12
-351	val_351	2010-08-15	12
-255	val_255	2010-08-15	12
-104	val_104	2010-08-15	12
-70	val_70	2010-08-15	12
-163	val_163	2010-08-15	12
-438	val_438	2010-08-15	12
-119	val_119	2010-08-15	12
-414	val_414	2010-08-15	12
-200	val_200	2010-08-15	12
-491	val_491	2010-08-15	12
-237	val_237	2010-08-15	12
-439	val_439	2010-08-15	12
-360	val_360	2010-08-15	12
-248	val_248	2010-08-15	12
-479	val_479	2010-08-15	12
-305	val_305	2010-08-15	12
-417	val_417	2010-08-15	12
-199	val_199	2010-08-15	12
-444	val_444	2010-08-15	12
-120	val_120	2010-08-15	12
-429	val_429	2010-08-15	12
-169	val_169	2010-08-15	12
-443	val_443	2010-08-15	12
-323	val_323	2010-08-15	12
-325	val_325	2010-08-15	12
-277	val_277	2010-08-15	12
-230	val_230	2010-08-15	12
-478	val_478	2010-08-15	12
-178	val_178	2010-08-15	12
-468	val_468	2010-08-15	12
-310	val_310	2010-08-15	12
-317	val_317	2010-08-15	12
-333	val_333	2010-08-15	12
-493	val_493	2010-08-15	12
-460	val_460	2010-08-15	12
-207	val_207	2010-08-15	12
-249	val_249	2010-08-15	12
-265	val_265	2010-08-15	12
-480	val_480	2010-08-15	12
-83	val_83	2010-08-15	12
-136	val_136	2010-08-15	12
-353	val_353	2010-08-15	12
-172	val_172	2010-08-15	12
-214	val_214	2010-08-15	12
-462	val_462	2010-08-15	12
-233	val_233	2010-08-15	12
-406	val_406	2010-08-15	12
-133	val_133	2010-08-15	12
-175	val_175	2010-08-15	12
-189	val_189	2010-08-15	12
-454	val_454	2010-08-15	12
-375	val_375	2010-08-15	12
-401	val_401	2010-08-15	12
-421	val_421	2010-08-15	12
-407	val_407	2010-08-15	12
-384	val_384	2010-08-15	12
-256	val_256	2010-08-15	12
-26	val_26	2010-08-15	12
-134	val_134	2010-08-15	12
-67	val_67	2010-08-15	12
-384	val_384	2010-08-15	12
-379	val_379	2010-08-15	12
-18	val_18	2010-08-15	12
-462	val_462	2010-08-15	12
-492	val_492	2010-08-15	12
-100	val_100	2010-08-15	12
-298	val_298	2010-08-15	12
-9	val_9	2010-08-15	12
-341	val_341	2010-08-15	12
-498	val_498	2010-08-15	12
-146	val_146	2010-08-15	12
-458	val_458	2010-08-15	12
-362	val_362	2010-08-15	12
-186	val_186	2010-08-15	12
-285	val_285	2010-08-15	12
-348	val_348	2010-08-15	12
-167	val_167	2010-08-15	12
-18	val_18	2010-08-15	12
-273	val_273	2010-08-15	12
-183	val_183	2010-08-15	12
-281	val_281	2010-08-15	12
-344	val_344	2010-08-15	12
-97	val_97	2010-08-15	12
-469	val_469	2010-08-15	12
-315	val_315	2010-08-15	12
-84	val_84	2010-08-15	12
-28	val_28	2010-08-15	12
-37	val_37	2010-08-15	12
-448	val_448	2010-08-15	12
-152	val_152	2010-08-15	12
-348	val_348	2010-08-15	12
-307	val_307	2010-08-15	12
-194	val_194	2010-08-15	12
-414	val_414	2010-08-15	12
-477	val_477	2010-08-15	12
-222	val_222	2010-08-15	12
-126	val_126	2010-08-15	12
-90	val_90	2010-08-15	12
-169	val_169	2010-08-15	12
-403	val_403	2010-08-15	12
-400	val_400	2010-08-15	12
-200	val_200	2010-08-15	12
-97	val_97	2010-08-15	12
diff --git a/sql/hive/src/test/resources/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 b/sql/hive/src/test/resources/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da b/sql/hive/src/test/resources/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92
deleted file mode 100644
index a4c81ff9f99cd3392c59fa86cc2aa925f220dca2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92
+++ /dev/null
@@ -1,500 +0,0 @@
-0	val_0	2008-04-08	11
-0	val_0	2008-04-08	11
-0	val_0	2008-04-08	11
-2	val_2	2008-04-08	11
-4	val_4	2008-04-08	11
-5	val_5	2008-04-08	11
-5	val_5	2008-04-08	11
-5	val_5	2008-04-08	11
-8	val_8	2008-04-08	11
-9	val_9	2008-04-08	11
-10	val_10	2008-04-08	11
-11	val_11	2008-04-08	11
-12	val_12	2008-04-08	11
-12	val_12	2008-04-08	11
-15	val_15	2008-04-08	11
-15	val_15	2008-04-08	11
-17	val_17	2008-04-08	11
-18	val_18	2008-04-08	11
-18	val_18	2008-04-08	11
-19	val_19	2008-04-08	11
-20	val_20	2008-04-08	11
-24	val_24	2008-04-08	11
-24	val_24	2008-04-08	11
-26	val_26	2008-04-08	11
-26	val_26	2008-04-08	11
-27	val_27	2008-04-08	11
-28	val_28	2008-04-08	11
-30	val_30	2008-04-08	11
-33	val_33	2008-04-08	11
-34	val_34	2008-04-08	11
-35	val_35	2008-04-08	11
-35	val_35	2008-04-08	11
-35	val_35	2008-04-08	11
-37	val_37	2008-04-08	11
-37	val_37	2008-04-08	11
-41	val_41	2008-04-08	11
-42	val_42	2008-04-08	11
-42	val_42	2008-04-08	11
-43	val_43	2008-04-08	11
-44	val_44	2008-04-08	11
-47	val_47	2008-04-08	11
-51	val_51	2008-04-08	11
-51	val_51	2008-04-08	11
-53	val_53	2008-04-08	11
-54	val_54	2008-04-08	11
-57	val_57	2008-04-08	11
-58	val_58	2008-04-08	11
-58	val_58	2008-04-08	11
-64	val_64	2008-04-08	11
-65	val_65	2008-04-08	11
-66	val_66	2008-04-08	11
-67	val_67	2008-04-08	11
-67	val_67	2008-04-08	11
-69	val_69	2008-04-08	11
-70	val_70	2008-04-08	11
-70	val_70	2008-04-08	11
-70	val_70	2008-04-08	11
-72	val_72	2008-04-08	11
-72	val_72	2008-04-08	11
-74	val_74	2008-04-08	11
-76	val_76	2008-04-08	11
-76	val_76	2008-04-08	11
-77	val_77	2008-04-08	11
-78	val_78	2008-04-08	11
-80	val_80	2008-04-08	11
-82	val_82	2008-04-08	11
-83	val_83	2008-04-08	11
-83	val_83	2008-04-08	11
-84	val_84	2008-04-08	11
-84	val_84	2008-04-08	11
-85	val_85	2008-04-08	11
-86	val_86	2008-04-08	11
-87	val_87	2008-04-08	11
-90	val_90	2008-04-08	11
-90	val_90	2008-04-08	11
-90	val_90	2008-04-08	11
-92	val_92	2008-04-08	11
-95	val_95	2008-04-08	11
-95	val_95	2008-04-08	11
-96	val_96	2008-04-08	11
-97	val_97	2008-04-08	11
-97	val_97	2008-04-08	11
-98	val_98	2008-04-08	11
-98	val_98	2008-04-08	11
-100	val_100	2008-04-08	11
-100	val_100	2008-04-08	11
-103	val_103	2008-04-08	11
-103	val_103	2008-04-08	11
-104	val_104	2008-04-08	11
-104	val_104	2008-04-08	11
-105	val_105	2008-04-08	11
-111	val_111	2008-04-08	11
-113	val_113	2008-04-08	11
-113	val_113	2008-04-08	11
-114	val_114	2008-04-08	11
-116	val_116	2008-04-08	11
-118	val_118	2008-04-08	11
-118	val_118	2008-04-08	11
-119	val_119	2008-04-08	11
-119	val_119	2008-04-08	11
-119	val_119	2008-04-08	11
-120	val_120	2008-04-08	11
-120	val_120	2008-04-08	11
-125	val_125	2008-04-08	11
-125	val_125	2008-04-08	11
-126	val_126	2008-04-08	11
-128	val_128	2008-04-08	11
-128	val_128	2008-04-08	11
-128	val_128	2008-04-08	11
-129	val_129	2008-04-08	11
-129	val_129	2008-04-08	11
-131	val_131	2008-04-08	11
-133	val_133	2008-04-08	11
-134	val_134	2008-04-08	11
-134	val_134	2008-04-08	11
-136	val_136	2008-04-08	11
-137	val_137	2008-04-08	11
-137	val_137	2008-04-08	11
-138	val_138	2008-04-08	11
-138	val_138	2008-04-08	11
-138	val_138	2008-04-08	11
-138	val_138	2008-04-08	11
-143	val_143	2008-04-08	11
-145	val_145	2008-04-08	11
-146	val_146	2008-04-08	11
-146	val_146	2008-04-08	11
-149	val_149	2008-04-08	11
-149	val_149	2008-04-08	11
-150	val_150	2008-04-08	11
-152	val_152	2008-04-08	11
-152	val_152	2008-04-08	11
-153	val_153	2008-04-08	11
-155	val_155	2008-04-08	11
-156	val_156	2008-04-08	11
-157	val_157	2008-04-08	11
-158	val_158	2008-04-08	11
-160	val_160	2008-04-08	11
-162	val_162	2008-04-08	11
-163	val_163	2008-04-08	11
-164	val_164	2008-04-08	11
-164	val_164	2008-04-08	11
-165	val_165	2008-04-08	11
-165	val_165	2008-04-08	11
-166	val_166	2008-04-08	11
-167	val_167	2008-04-08	11
-167	val_167	2008-04-08	11
-167	val_167	2008-04-08	11
-168	val_168	2008-04-08	11
-169	val_169	2008-04-08	11
-169	val_169	2008-04-08	11
-169	val_169	2008-04-08	11
-169	val_169	2008-04-08	11
-170	val_170	2008-04-08	11
-172	val_172	2008-04-08	11
-172	val_172	2008-04-08	11
-174	val_174	2008-04-08	11
-174	val_174	2008-04-08	11
-175	val_175	2008-04-08	11
-175	val_175	2008-04-08	11
-176	val_176	2008-04-08	11
-176	val_176	2008-04-08	11
-177	val_177	2008-04-08	11
-178	val_178	2008-04-08	11
-179	val_179	2008-04-08	11
-179	val_179	2008-04-08	11
-180	val_180	2008-04-08	11
-181	val_181	2008-04-08	11
-183	val_183	2008-04-08	11
-186	val_186	2008-04-08	11
-187	val_187	2008-04-08	11
-187	val_187	2008-04-08	11
-187	val_187	2008-04-08	11
-189	val_189	2008-04-08	11
-190	val_190	2008-04-08	11
-191	val_191	2008-04-08	11
-191	val_191	2008-04-08	11
-192	val_192	2008-04-08	11
-193	val_193	2008-04-08	11
-193	val_193	2008-04-08	11
-193	val_193	2008-04-08	11
-194	val_194	2008-04-08	11
-195	val_195	2008-04-08	11
-195	val_195	2008-04-08	11
-196	val_196	2008-04-08	11
-197	val_197	2008-04-08	11
-197	val_197	2008-04-08	11
-199	val_199	2008-04-08	11
-199	val_199	2008-04-08	11
-199	val_199	2008-04-08	11
-200	val_200	2008-04-08	11
-200	val_200	2008-04-08	11
-201	val_201	2008-04-08	11
-202	val_202	2008-04-08	11
-203	val_203	2008-04-08	11
-203	val_203	2008-04-08	11
-205	val_205	2008-04-08	11
-205	val_205	2008-04-08	11
-207	val_207	2008-04-08	11
-207	val_207	2008-04-08	11
-208	val_208	2008-04-08	11
-208	val_208	2008-04-08	11
-208	val_208	2008-04-08	11
-209	val_209	2008-04-08	11
-209	val_209	2008-04-08	11
-213	val_213	2008-04-08	11
-213	val_213	2008-04-08	11
-214	val_214	2008-04-08	11
-216	val_216	2008-04-08	11
-216	val_216	2008-04-08	11
-217	val_217	2008-04-08	11
-217	val_217	2008-04-08	11
-218	val_218	2008-04-08	11
-219	val_219	2008-04-08	11
-219	val_219	2008-04-08	11
-221	val_221	2008-04-08	11
-221	val_221	2008-04-08	11
-222	val_222	2008-04-08	11
-223	val_223	2008-04-08	11
-223	val_223	2008-04-08	11
-224	val_224	2008-04-08	11
-224	val_224	2008-04-08	11
-226	val_226	2008-04-08	11
-228	val_228	2008-04-08	11
-229	val_229	2008-04-08	11
-229	val_229	2008-04-08	11
-230	val_230	2008-04-08	11
-230	val_230	2008-04-08	11
-230	val_230	2008-04-08	11
-230	val_230	2008-04-08	11
-230	val_230	2008-04-08	11
-233	val_233	2008-04-08	11
-233	val_233	2008-04-08	11
-235	val_235	2008-04-08	11
-237	val_237	2008-04-08	11
-237	val_237	2008-04-08	11
-238	val_238	2008-04-08	11
-238	val_238	2008-04-08	11
-239	val_239	2008-04-08	11
-239	val_239	2008-04-08	11
-241	val_241	2008-04-08	11
-242	val_242	2008-04-08	11
-242	val_242	2008-04-08	11
-244	val_244	2008-04-08	11
-247	val_247	2008-04-08	11
-248	val_248	2008-04-08	11
-249	val_249	2008-04-08	11
-252	val_252	2008-04-08	11
-255	val_255	2008-04-08	11
-255	val_255	2008-04-08	11
-256	val_256	2008-04-08	11
-256	val_256	2008-04-08	11
-257	val_257	2008-04-08	11
-258	val_258	2008-04-08	11
-260	val_260	2008-04-08	11
-262	val_262	2008-04-08	11
-263	val_263	2008-04-08	11
-265	val_265	2008-04-08	11
-265	val_265	2008-04-08	11
-266	val_266	2008-04-08	11
-272	val_272	2008-04-08	11
-272	val_272	2008-04-08	11
-273	val_273	2008-04-08	11
-273	val_273	2008-04-08	11
-273	val_273	2008-04-08	11
-274	val_274	2008-04-08	11
-275	val_275	2008-04-08	11
-277	val_277	2008-04-08	11
-277	val_277	2008-04-08	11
-277	val_277	2008-04-08	11
-277	val_277	2008-04-08	11
-278	val_278	2008-04-08	11
-278	val_278	2008-04-08	11
-280	val_280	2008-04-08	11
-280	val_280	2008-04-08	11
-281	val_281	2008-04-08	11
-281	val_281	2008-04-08	11
-282	val_282	2008-04-08	11
-282	val_282	2008-04-08	11
-283	val_283	2008-04-08	11
-284	val_284	2008-04-08	11
-285	val_285	2008-04-08	11
-286	val_286	2008-04-08	11
-287	val_287	2008-04-08	11
-288	val_288	2008-04-08	11
-288	val_288	2008-04-08	11
-289	val_289	2008-04-08	11
-291	val_291	2008-04-08	11
-292	val_292	2008-04-08	11
-296	val_296	2008-04-08	11
-298	val_298	2008-04-08	11
-298	val_298	2008-04-08	11
-298	val_298	2008-04-08	11
-302	val_302	2008-04-08	11
-305	val_305	2008-04-08	11
-306	val_306	2008-04-08	11
-307	val_307	2008-04-08	11
-307	val_307	2008-04-08	11
-308	val_308	2008-04-08	11
-309	val_309	2008-04-08	11
-309	val_309	2008-04-08	11
-310	val_310	2008-04-08	11
-311	val_311	2008-04-08	11
-311	val_311	2008-04-08	11
-311	val_311	2008-04-08	11
-315	val_315	2008-04-08	11
-316	val_316	2008-04-08	11
-316	val_316	2008-04-08	11
-316	val_316	2008-04-08	11
-317	val_317	2008-04-08	11
-317	val_317	2008-04-08	11
-318	val_318	2008-04-08	11
-318	val_318	2008-04-08	11
-318	val_318	2008-04-08	11
-321	val_321	2008-04-08	11
-321	val_321	2008-04-08	11
-322	val_322	2008-04-08	11
-322	val_322	2008-04-08	11
-323	val_323	2008-04-08	11
-325	val_325	2008-04-08	11
-325	val_325	2008-04-08	11
-327	val_327	2008-04-08	11
-327	val_327	2008-04-08	11
-327	val_327	2008-04-08	11
-331	val_331	2008-04-08	11
-331	val_331	2008-04-08	11
-332	val_332	2008-04-08	11
-333	val_333	2008-04-08	11
-333	val_333	2008-04-08	11
-335	val_335	2008-04-08	11
-336	val_336	2008-04-08	11
-338	val_338	2008-04-08	11
-339	val_339	2008-04-08	11
-341	val_341	2008-04-08	11
-342	val_342	2008-04-08	11
-342	val_342	2008-04-08	11
-344	val_344	2008-04-08	11
-344	val_344	2008-04-08	11
-345	val_345	2008-04-08	11
-348	val_348	2008-04-08	11
-348	val_348	2008-04-08	11
-348	val_348	2008-04-08	11
-348	val_348	2008-04-08	11
-348	val_348	2008-04-08	11
-351	val_351	2008-04-08	11
-353	val_353	2008-04-08	11
-353	val_353	2008-04-08	11
-356	val_356	2008-04-08	11
-360	val_360	2008-04-08	11
-362	val_362	2008-04-08	11
-364	val_364	2008-04-08	11
-365	val_365	2008-04-08	11
-366	val_366	2008-04-08	11
-367	val_367	2008-04-08	11
-367	val_367	2008-04-08	11
-368	val_368	2008-04-08	11
-369	val_369	2008-04-08	11
-369	val_369	2008-04-08	11
-369	val_369	2008-04-08	11
-373	val_373	2008-04-08	11
-374	val_374	2008-04-08	11
-375	val_375	2008-04-08	11
-377	val_377	2008-04-08	11
-378	val_378	2008-04-08	11
-379	val_379	2008-04-08	11
-382	val_382	2008-04-08	11
-382	val_382	2008-04-08	11
-384	val_384	2008-04-08	11
-384	val_384	2008-04-08	11
-384	val_384	2008-04-08	11
-386	val_386	2008-04-08	11
-389	val_389	2008-04-08	11
-392	val_392	2008-04-08	11
-393	val_393	2008-04-08	11
-394	val_394	2008-04-08	11
-395	val_395	2008-04-08	11
-395	val_395	2008-04-08	11
-396	val_396	2008-04-08	11
-396	val_396	2008-04-08	11
-396	val_396	2008-04-08	11
-397	val_397	2008-04-08	11
-397	val_397	2008-04-08	11
-399	val_399	2008-04-08	11
-399	val_399	2008-04-08	11
-400	val_400	2008-04-08	11
-401	val_401	2008-04-08	11
-401	val_401	2008-04-08	11
-401	val_401	2008-04-08	11
-401	val_401	2008-04-08	11
-401	val_401	2008-04-08	11
-402	val_402	2008-04-08	11
-403	val_403	2008-04-08	11
-403	val_403	2008-04-08	11
-403	val_403	2008-04-08	11
-404	val_404	2008-04-08	11
-404	val_404	2008-04-08	11
-406	val_406	2008-04-08	11
-406	val_406	2008-04-08	11
-406	val_406	2008-04-08	11
-406	val_406	2008-04-08	11
-407	val_407	2008-04-08	11
-409	val_409	2008-04-08	11
-409	val_409	2008-04-08	11
-409	val_409	2008-04-08	11
-411	val_411	2008-04-08	11
-413	val_413	2008-04-08	11
-413	val_413	2008-04-08	11
-414	val_414	2008-04-08	11
-414	val_414	2008-04-08	11
-417	val_417	2008-04-08	11
-417	val_417	2008-04-08	11
-417	val_417	2008-04-08	11
-418	val_418	2008-04-08	11
-419	val_419	2008-04-08	11
-421	val_421	2008-04-08	11
-424	val_424	2008-04-08	11
-424	val_424	2008-04-08	11
-427	val_427	2008-04-08	11
-429	val_429	2008-04-08	11
-429	val_429	2008-04-08	11
-430	val_430	2008-04-08	11
-430	val_430	2008-04-08	11
-430	val_430	2008-04-08	11
-431	val_431	2008-04-08	11
-431	val_431	2008-04-08	11
-431	val_431	2008-04-08	11
-432	val_432	2008-04-08	11
-435	val_435	2008-04-08	11
-436	val_436	2008-04-08	11
-437	val_437	2008-04-08	11
-438	val_438	2008-04-08	11
-438	val_438	2008-04-08	11
-438	val_438	2008-04-08	11
-439	val_439	2008-04-08	11
-439	val_439	2008-04-08	11
-443	val_443	2008-04-08	11
-444	val_444	2008-04-08	11
-446	val_446	2008-04-08	11
-448	val_448	2008-04-08	11
-449	val_449	2008-04-08	11
-452	val_452	2008-04-08	11
-453	val_453	2008-04-08	11
-454	val_454	2008-04-08	11
-454	val_454	2008-04-08	11
-454	val_454	2008-04-08	11
-455	val_455	2008-04-08	11
-457	val_457	2008-04-08	11
-458	val_458	2008-04-08	11
-458	val_458	2008-04-08	11
-459	val_459	2008-04-08	11
-459	val_459	2008-04-08	11
-460	val_460	2008-04-08	11
-462	val_462	2008-04-08	11
-462	val_462	2008-04-08	11
-463	val_463	2008-04-08	11
-463	val_463	2008-04-08	11
-466	val_466	2008-04-08	11
-466	val_466	2008-04-08	11
-466	val_466	2008-04-08	11
-467	val_467	2008-04-08	11
-468	val_468	2008-04-08	11
-468	val_468	2008-04-08	11
-468	val_468	2008-04-08	11
-468	val_468	2008-04-08	11
-469	val_469	2008-04-08	11
-469	val_469	2008-04-08	11
-469	val_469	2008-04-08	11
-469	val_469	2008-04-08	11
-469	val_469	2008-04-08	11
-470	val_470	2008-04-08	11
-472	val_472	2008-04-08	11
-475	val_475	2008-04-08	11
-477	val_477	2008-04-08	11
-478	val_478	2008-04-08	11
-478	val_478	2008-04-08	11
-479	val_479	2008-04-08	11
-480	val_480	2008-04-08	11
-480	val_480	2008-04-08	11
-480	val_480	2008-04-08	11
-481	val_481	2008-04-08	11
-482	val_482	2008-04-08	11
-483	val_483	2008-04-08	11
-484	val_484	2008-04-08	11
-485	val_485	2008-04-08	11
-487	val_487	2008-04-08	11
-489	val_489	2008-04-08	11
-489	val_489	2008-04-08	11
-489	val_489	2008-04-08	11
-489	val_489	2008-04-08	11
-490	val_490	2008-04-08	11
-491	val_491	2008-04-08	11
-492	val_492	2008-04-08	11
-492	val_492	2008-04-08	11
-493	val_493	2008-04-08	11
-494	val_494	2008-04-08	11
-495	val_495	2008-04-08	11
-496	val_496	2008-04-08	11
-497	val_497	2008-04-08	11
-498	val_498	2008-04-08	11
-498	val_498	2008-04-08	11
-498	val_498	2008-04-08	11
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858
deleted file mode 100644
index d66ce5d097ce627635235fe0ab634a06d7ab9095..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:merge_dynamic_part
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1431818582215388621/merge_dynamic_part
-inputformat:org.apache.hadoop.mapred.TextInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-columns:struct columns { i32 key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string ds, string hr}
-totalNumberFiles:4
-totalFileSize:5812
-maxFileSize:1612
-minFileSize:1358
-lastAccessTime:0
-lastUpdateTime:1389738875000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92
deleted file mode 100644
index a4c81ff9f99cd3392c59fa86cc2aa925f220dca2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92
+++ /dev/null
@@ -1,500 +0,0 @@
-0	val_0	2008-04-08	11
-0	val_0	2008-04-08	11
-0	val_0	2008-04-08	11
-2	val_2	2008-04-08	11
-4	val_4	2008-04-08	11
-5	val_5	2008-04-08	11
-5	val_5	2008-04-08	11
-5	val_5	2008-04-08	11
-8	val_8	2008-04-08	11
-9	val_9	2008-04-08	11
-10	val_10	2008-04-08	11
-11	val_11	2008-04-08	11
-12	val_12	2008-04-08	11
-12	val_12	2008-04-08	11
-15	val_15	2008-04-08	11
-15	val_15	2008-04-08	11
-17	val_17	2008-04-08	11
-18	val_18	2008-04-08	11
-18	val_18	2008-04-08	11
-19	val_19	2008-04-08	11
-20	val_20	2008-04-08	11
-24	val_24	2008-04-08	11
-24	val_24	2008-04-08	11
-26	val_26	2008-04-08	11
-26	val_26	2008-04-08	11
-27	val_27	2008-04-08	11
-28	val_28	2008-04-08	11
-30	val_30	2008-04-08	11
-33	val_33	2008-04-08	11
-34	val_34	2008-04-08	11
-35	val_35	2008-04-08	11
-35	val_35	2008-04-08	11
-35	val_35	2008-04-08	11
-37	val_37	2008-04-08	11
-37	val_37	2008-04-08	11
-41	val_41	2008-04-08	11
-42	val_42	2008-04-08	11
-42	val_42	2008-04-08	11
-43	val_43	2008-04-08	11
-44	val_44	2008-04-08	11
-47	val_47	2008-04-08	11
-51	val_51	2008-04-08	11
-51	val_51	2008-04-08	11
-53	val_53	2008-04-08	11
-54	val_54	2008-04-08	11
-57	val_57	2008-04-08	11
-58	val_58	2008-04-08	11
-58	val_58	2008-04-08	11
-64	val_64	2008-04-08	11
-65	val_65	2008-04-08	11
-66	val_66	2008-04-08	11
-67	val_67	2008-04-08	11
-67	val_67	2008-04-08	11
-69	val_69	2008-04-08	11
-70	val_70	2008-04-08	11
-70	val_70	2008-04-08	11
-70	val_70	2008-04-08	11
-72	val_72	2008-04-08	11
-72	val_72	2008-04-08	11
-74	val_74	2008-04-08	11
-76	val_76	2008-04-08	11
-76	val_76	2008-04-08	11
-77	val_77	2008-04-08	11
-78	val_78	2008-04-08	11
-80	val_80	2008-04-08	11
-82	val_82	2008-04-08	11
-83	val_83	2008-04-08	11
-83	val_83	2008-04-08	11
-84	val_84	2008-04-08	11
-84	val_84	2008-04-08	11
-85	val_85	2008-04-08	11
-86	val_86	2008-04-08	11
-87	val_87	2008-04-08	11
-90	val_90	2008-04-08	11
-90	val_90	2008-04-08	11
-90	val_90	2008-04-08	11
-92	val_92	2008-04-08	11
-95	val_95	2008-04-08	11
-95	val_95	2008-04-08	11
-96	val_96	2008-04-08	11
-97	val_97	2008-04-08	11
-97	val_97	2008-04-08	11
-98	val_98	2008-04-08	11
-98	val_98	2008-04-08	11
-100	val_100	2008-04-08	11
-100	val_100	2008-04-08	11
-103	val_103	2008-04-08	11
-103	val_103	2008-04-08	11
-104	val_104	2008-04-08	11
-104	val_104	2008-04-08	11
-105	val_105	2008-04-08	11
-111	val_111	2008-04-08	11
-113	val_113	2008-04-08	11
-113	val_113	2008-04-08	11
-114	val_114	2008-04-08	11
-116	val_116	2008-04-08	11
-118	val_118	2008-04-08	11
-118	val_118	2008-04-08	11
-119	val_119	2008-04-08	11
-119	val_119	2008-04-08	11
-119	val_119	2008-04-08	11
-120	val_120	2008-04-08	11
-120	val_120	2008-04-08	11
-125	val_125	2008-04-08	11
-125	val_125	2008-04-08	11
-126	val_126	2008-04-08	11
-128	val_128	2008-04-08	11
-128	val_128	2008-04-08	11
-128	val_128	2008-04-08	11
-129	val_129	2008-04-08	11
-129	val_129	2008-04-08	11
-131	val_131	2008-04-08	11
-133	val_133	2008-04-08	11
-134	val_134	2008-04-08	11
-134	val_134	2008-04-08	11
-136	val_136	2008-04-08	11
-137	val_137	2008-04-08	11
-137	val_137	2008-04-08	11
-138	val_138	2008-04-08	11
-138	val_138	2008-04-08	11
-138	val_138	2008-04-08	11
-138	val_138	2008-04-08	11
-143	val_143	2008-04-08	11
-145	val_145	2008-04-08	11
-146	val_146	2008-04-08	11
-146	val_146	2008-04-08	11
-149	val_149	2008-04-08	11
-149	val_149	2008-04-08	11
-150	val_150	2008-04-08	11
-152	val_152	2008-04-08	11
-152	val_152	2008-04-08	11
-153	val_153	2008-04-08	11
-155	val_155	2008-04-08	11
-156	val_156	2008-04-08	11
-157	val_157	2008-04-08	11
-158	val_158	2008-04-08	11
-160	val_160	2008-04-08	11
-162	val_162	2008-04-08	11
-163	val_163	2008-04-08	11
-164	val_164	2008-04-08	11
-164	val_164	2008-04-08	11
-165	val_165	2008-04-08	11
-165	val_165	2008-04-08	11
-166	val_166	2008-04-08	11
-167	val_167	2008-04-08	11
-167	val_167	2008-04-08	11
-167	val_167	2008-04-08	11
-168	val_168	2008-04-08	11
-169	val_169	2008-04-08	11
-169	val_169	2008-04-08	11
-169	val_169	2008-04-08	11
-169	val_169	2008-04-08	11
-170	val_170	2008-04-08	11
-172	val_172	2008-04-08	11
-172	val_172	2008-04-08	11
-174	val_174	2008-04-08	11
-174	val_174	2008-04-08	11
-175	val_175	2008-04-08	11
-175	val_175	2008-04-08	11
-176	val_176	2008-04-08	11
-176	val_176	2008-04-08	11
-177	val_177	2008-04-08	11
-178	val_178	2008-04-08	11
-179	val_179	2008-04-08	11
-179	val_179	2008-04-08	11
-180	val_180	2008-04-08	11
-181	val_181	2008-04-08	11
-183	val_183	2008-04-08	11
-186	val_186	2008-04-08	11
-187	val_187	2008-04-08	11
-187	val_187	2008-04-08	11
-187	val_187	2008-04-08	11
-189	val_189	2008-04-08	11
-190	val_190	2008-04-08	11
-191	val_191	2008-04-08	11
-191	val_191	2008-04-08	11
-192	val_192	2008-04-08	11
-193	val_193	2008-04-08	11
-193	val_193	2008-04-08	11
-193	val_193	2008-04-08	11
-194	val_194	2008-04-08	11
-195	val_195	2008-04-08	11
-195	val_195	2008-04-08	11
-196	val_196	2008-04-08	11
-197	val_197	2008-04-08	11
-197	val_197	2008-04-08	11
-199	val_199	2008-04-08	11
-199	val_199	2008-04-08	11
-199	val_199	2008-04-08	11
-200	val_200	2008-04-08	11
-200	val_200	2008-04-08	11
-201	val_201	2008-04-08	11
-202	val_202	2008-04-08	11
-203	val_203	2008-04-08	11
-203	val_203	2008-04-08	11
-205	val_205	2008-04-08	11
-205	val_205	2008-04-08	11
-207	val_207	2008-04-08	11
-207	val_207	2008-04-08	11
-208	val_208	2008-04-08	11
-208	val_208	2008-04-08	11
-208	val_208	2008-04-08	11
-209	val_209	2008-04-08	11
-209	val_209	2008-04-08	11
-213	val_213	2008-04-08	11
-213	val_213	2008-04-08	11
-214	val_214	2008-04-08	11
-216	val_216	2008-04-08	11
-216	val_216	2008-04-08	11
-217	val_217	2008-04-08	11
-217	val_217	2008-04-08	11
-218	val_218	2008-04-08	11
-219	val_219	2008-04-08	11
-219	val_219	2008-04-08	11
-221	val_221	2008-04-08	11
-221	val_221	2008-04-08	11
-222	val_222	2008-04-08	11
-223	val_223	2008-04-08	11
-223	val_223	2008-04-08	11
-224	val_224	2008-04-08	11
-224	val_224	2008-04-08	11
-226	val_226	2008-04-08	11
-228	val_228	2008-04-08	11
-229	val_229	2008-04-08	11
-229	val_229	2008-04-08	11
-230	val_230	2008-04-08	11
-230	val_230	2008-04-08	11
-230	val_230	2008-04-08	11
-230	val_230	2008-04-08	11
-230	val_230	2008-04-08	11
-233	val_233	2008-04-08	11
-233	val_233	2008-04-08	11
-235	val_235	2008-04-08	11
-237	val_237	2008-04-08	11
-237	val_237	2008-04-08	11
-238	val_238	2008-04-08	11
-238	val_238	2008-04-08	11
-239	val_239	2008-04-08	11
-239	val_239	2008-04-08	11
-241	val_241	2008-04-08	11
-242	val_242	2008-04-08	11
-242	val_242	2008-04-08	11
-244	val_244	2008-04-08	11
-247	val_247	2008-04-08	11
-248	val_248	2008-04-08	11
-249	val_249	2008-04-08	11
-252	val_252	2008-04-08	11
-255	val_255	2008-04-08	11
-255	val_255	2008-04-08	11
-256	val_256	2008-04-08	11
-256	val_256	2008-04-08	11
-257	val_257	2008-04-08	11
-258	val_258	2008-04-08	11
-260	val_260	2008-04-08	11
-262	val_262	2008-04-08	11
-263	val_263	2008-04-08	11
-265	val_265	2008-04-08	11
-265	val_265	2008-04-08	11
-266	val_266	2008-04-08	11
-272	val_272	2008-04-08	11
-272	val_272	2008-04-08	11
-273	val_273	2008-04-08	11
-273	val_273	2008-04-08	11
-273	val_273	2008-04-08	11
-274	val_274	2008-04-08	11
-275	val_275	2008-04-08	11
-277	val_277	2008-04-08	11
-277	val_277	2008-04-08	11
-277	val_277	2008-04-08	11
-277	val_277	2008-04-08	11
-278	val_278	2008-04-08	11
-278	val_278	2008-04-08	11
-280	val_280	2008-04-08	11
-280	val_280	2008-04-08	11
-281	val_281	2008-04-08	11
-281	val_281	2008-04-08	11
-282	val_282	2008-04-08	11
-282	val_282	2008-04-08	11
-283	val_283	2008-04-08	11
-284	val_284	2008-04-08	11
-285	val_285	2008-04-08	11
-286	val_286	2008-04-08	11
-287	val_287	2008-04-08	11
-288	val_288	2008-04-08	11
-288	val_288	2008-04-08	11
-289	val_289	2008-04-08	11
-291	val_291	2008-04-08	11
-292	val_292	2008-04-08	11
-296	val_296	2008-04-08	11
-298	val_298	2008-04-08	11
-298	val_298	2008-04-08	11
-298	val_298	2008-04-08	11
-302	val_302	2008-04-08	11
-305	val_305	2008-04-08	11
-306	val_306	2008-04-08	11
-307	val_307	2008-04-08	11
-307	val_307	2008-04-08	11
-308	val_308	2008-04-08	11
-309	val_309	2008-04-08	11
-309	val_309	2008-04-08	11
-310	val_310	2008-04-08	11
-311	val_311	2008-04-08	11
-311	val_311	2008-04-08	11
-311	val_311	2008-04-08	11
-315	val_315	2008-04-08	11
-316	val_316	2008-04-08	11
-316	val_316	2008-04-08	11
-316	val_316	2008-04-08	11
-317	val_317	2008-04-08	11
-317	val_317	2008-04-08	11
-318	val_318	2008-04-08	11
-318	val_318	2008-04-08	11
-318	val_318	2008-04-08	11
-321	val_321	2008-04-08	11
-321	val_321	2008-04-08	11
-322	val_322	2008-04-08	11
-322	val_322	2008-04-08	11
-323	val_323	2008-04-08	11
-325	val_325	2008-04-08	11
-325	val_325	2008-04-08	11
-327	val_327	2008-04-08	11
-327	val_327	2008-04-08	11
-327	val_327	2008-04-08	11
-331	val_331	2008-04-08	11
-331	val_331	2008-04-08	11
-332	val_332	2008-04-08	11
-333	val_333	2008-04-08	11
-333	val_333	2008-04-08	11
-335	val_335	2008-04-08	11
-336	val_336	2008-04-08	11
-338	val_338	2008-04-08	11
-339	val_339	2008-04-08	11
-341	val_341	2008-04-08	11
-342	val_342	2008-04-08	11
-342	val_342	2008-04-08	11
-344	val_344	2008-04-08	11
-344	val_344	2008-04-08	11
-345	val_345	2008-04-08	11
-348	val_348	2008-04-08	11
-348	val_348	2008-04-08	11
-348	val_348	2008-04-08	11
-348	val_348	2008-04-08	11
-348	val_348	2008-04-08	11
-351	val_351	2008-04-08	11
-353	val_353	2008-04-08	11
-353	val_353	2008-04-08	11
-356	val_356	2008-04-08	11
-360	val_360	2008-04-08	11
-362	val_362	2008-04-08	11
-364	val_364	2008-04-08	11
-365	val_365	2008-04-08	11
-366	val_366	2008-04-08	11
-367	val_367	2008-04-08	11
-367	val_367	2008-04-08	11
-368	val_368	2008-04-08	11
-369	val_369	2008-04-08	11
-369	val_369	2008-04-08	11
-369	val_369	2008-04-08	11
-373	val_373	2008-04-08	11
-374	val_374	2008-04-08	11
-375	val_375	2008-04-08	11
-377	val_377	2008-04-08	11
-378	val_378	2008-04-08	11
-379	val_379	2008-04-08	11
-382	val_382	2008-04-08	11
-382	val_382	2008-04-08	11
-384	val_384	2008-04-08	11
-384	val_384	2008-04-08	11
-384	val_384	2008-04-08	11
-386	val_386	2008-04-08	11
-389	val_389	2008-04-08	11
-392	val_392	2008-04-08	11
-393	val_393	2008-04-08	11
-394	val_394	2008-04-08	11
-395	val_395	2008-04-08	11
-395	val_395	2008-04-08	11
-396	val_396	2008-04-08	11
-396	val_396	2008-04-08	11
-396	val_396	2008-04-08	11
-397	val_397	2008-04-08	11
-397	val_397	2008-04-08	11
-399	val_399	2008-04-08	11
-399	val_399	2008-04-08	11
-400	val_400	2008-04-08	11
-401	val_401	2008-04-08	11
-401	val_401	2008-04-08	11
-401	val_401	2008-04-08	11
-401	val_401	2008-04-08	11
-401	val_401	2008-04-08	11
-402	val_402	2008-04-08	11
-403	val_403	2008-04-08	11
-403	val_403	2008-04-08	11
-403	val_403	2008-04-08	11
-404	val_404	2008-04-08	11
-404	val_404	2008-04-08	11
-406	val_406	2008-04-08	11
-406	val_406	2008-04-08	11
-406	val_406	2008-04-08	11
-406	val_406	2008-04-08	11
-407	val_407	2008-04-08	11
-409	val_409	2008-04-08	11
-409	val_409	2008-04-08	11
-409	val_409	2008-04-08	11
-411	val_411	2008-04-08	11
-413	val_413	2008-04-08	11
-413	val_413	2008-04-08	11
-414	val_414	2008-04-08	11
-414	val_414	2008-04-08	11
-417	val_417	2008-04-08	11
-417	val_417	2008-04-08	11
-417	val_417	2008-04-08	11
-418	val_418	2008-04-08	11
-419	val_419	2008-04-08	11
-421	val_421	2008-04-08	11
-424	val_424	2008-04-08	11
-424	val_424	2008-04-08	11
-427	val_427	2008-04-08	11
-429	val_429	2008-04-08	11
-429	val_429	2008-04-08	11
-430	val_430	2008-04-08	11
-430	val_430	2008-04-08	11
-430	val_430	2008-04-08	11
-431	val_431	2008-04-08	11
-431	val_431	2008-04-08	11
-431	val_431	2008-04-08	11
-432	val_432	2008-04-08	11
-435	val_435	2008-04-08	11
-436	val_436	2008-04-08	11
-437	val_437	2008-04-08	11
-438	val_438	2008-04-08	11
-438	val_438	2008-04-08	11
-438	val_438	2008-04-08	11
-439	val_439	2008-04-08	11
-439	val_439	2008-04-08	11
-443	val_443	2008-04-08	11
-444	val_444	2008-04-08	11
-446	val_446	2008-04-08	11
-448	val_448	2008-04-08	11
-449	val_449	2008-04-08	11
-452	val_452	2008-04-08	11
-453	val_453	2008-04-08	11
-454	val_454	2008-04-08	11
-454	val_454	2008-04-08	11
-454	val_454	2008-04-08	11
-455	val_455	2008-04-08	11
-457	val_457	2008-04-08	11
-458	val_458	2008-04-08	11
-458	val_458	2008-04-08	11
-459	val_459	2008-04-08	11
-459	val_459	2008-04-08	11
-460	val_460	2008-04-08	11
-462	val_462	2008-04-08	11
-462	val_462	2008-04-08	11
-463	val_463	2008-04-08	11
-463	val_463	2008-04-08	11
-466	val_466	2008-04-08	11
-466	val_466	2008-04-08	11
-466	val_466	2008-04-08	11
-467	val_467	2008-04-08	11
-468	val_468	2008-04-08	11
-468	val_468	2008-04-08	11
-468	val_468	2008-04-08	11
-468	val_468	2008-04-08	11
-469	val_469	2008-04-08	11
-469	val_469	2008-04-08	11
-469	val_469	2008-04-08	11
-469	val_469	2008-04-08	11
-469	val_469	2008-04-08	11
-470	val_470	2008-04-08	11
-472	val_472	2008-04-08	11
-475	val_475	2008-04-08	11
-477	val_477	2008-04-08	11
-478	val_478	2008-04-08	11
-478	val_478	2008-04-08	11
-479	val_479	2008-04-08	11
-480	val_480	2008-04-08	11
-480	val_480	2008-04-08	11
-480	val_480	2008-04-08	11
-481	val_481	2008-04-08	11
-482	val_482	2008-04-08	11
-483	val_483	2008-04-08	11
-484	val_484	2008-04-08	11
-485	val_485	2008-04-08	11
-487	val_487	2008-04-08	11
-489	val_489	2008-04-08	11
-489	val_489	2008-04-08	11
-489	val_489	2008-04-08	11
-489	val_489	2008-04-08	11
-490	val_490	2008-04-08	11
-491	val_491	2008-04-08	11
-492	val_492	2008-04-08	11
-492	val_492	2008-04-08	11
-493	val_493	2008-04-08	11
-494	val_494	2008-04-08	11
-495	val_495	2008-04-08	11
-496	val_496	2008-04-08	11
-497	val_497	2008-04-08	11
-498	val_498	2008-04-08	11
-498	val_498	2008-04-08	11
-498	val_498	2008-04-08	11
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858
deleted file mode 100644
index 06444f372bd60047f9a077d66bf050831bf324ef..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:merge_dynamic_part
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1431818582215388621/merge_dynamic_part
-inputformat:org.apache.hadoop.mapred.TextInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-columns:struct columns { i32 key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string ds, string hr}
-totalNumberFiles:1
-totalFileSize:5812
-maxFileSize:5812
-minFileSize:5812
-lastAccessTime:0
-lastUpdateTime:1389738910000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d b/sql/hive/src/test/resources/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f b/sql/hive/src/test/resources/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92
deleted file mode 100644
index a4c81ff9f99cd3392c59fa86cc2aa925f220dca2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92
+++ /dev/null
@@ -1,500 +0,0 @@
-0	val_0	2008-04-08	11
-0	val_0	2008-04-08	11
-0	val_0	2008-04-08	11
-2	val_2	2008-04-08	11
-4	val_4	2008-04-08	11
-5	val_5	2008-04-08	11
-5	val_5	2008-04-08	11
-5	val_5	2008-04-08	11
-8	val_8	2008-04-08	11
-9	val_9	2008-04-08	11
-10	val_10	2008-04-08	11
-11	val_11	2008-04-08	11
-12	val_12	2008-04-08	11
-12	val_12	2008-04-08	11
-15	val_15	2008-04-08	11
-15	val_15	2008-04-08	11
-17	val_17	2008-04-08	11
-18	val_18	2008-04-08	11
-18	val_18	2008-04-08	11
-19	val_19	2008-04-08	11
-20	val_20	2008-04-08	11
-24	val_24	2008-04-08	11
-24	val_24	2008-04-08	11
-26	val_26	2008-04-08	11
-26	val_26	2008-04-08	11
-27	val_27	2008-04-08	11
-28	val_28	2008-04-08	11
-30	val_30	2008-04-08	11
-33	val_33	2008-04-08	11
-34	val_34	2008-04-08	11
-35	val_35	2008-04-08	11
-35	val_35	2008-04-08	11
-35	val_35	2008-04-08	11
-37	val_37	2008-04-08	11
-37	val_37	2008-04-08	11
-41	val_41	2008-04-08	11
-42	val_42	2008-04-08	11
-42	val_42	2008-04-08	11
-43	val_43	2008-04-08	11
-44	val_44	2008-04-08	11
-47	val_47	2008-04-08	11
-51	val_51	2008-04-08	11
-51	val_51	2008-04-08	11
-53	val_53	2008-04-08	11
-54	val_54	2008-04-08	11
-57	val_57	2008-04-08	11
-58	val_58	2008-04-08	11
-58	val_58	2008-04-08	11
-64	val_64	2008-04-08	11
-65	val_65	2008-04-08	11
-66	val_66	2008-04-08	11
-67	val_67	2008-04-08	11
-67	val_67	2008-04-08	11
-69	val_69	2008-04-08	11
-70	val_70	2008-04-08	11
-70	val_70	2008-04-08	11
-70	val_70	2008-04-08	11
-72	val_72	2008-04-08	11
-72	val_72	2008-04-08	11
-74	val_74	2008-04-08	11
-76	val_76	2008-04-08	11
-76	val_76	2008-04-08	11
-77	val_77	2008-04-08	11
-78	val_78	2008-04-08	11
-80	val_80	2008-04-08	11
-82	val_82	2008-04-08	11
-83	val_83	2008-04-08	11
-83	val_83	2008-04-08	11
-84	val_84	2008-04-08	11
-84	val_84	2008-04-08	11
-85	val_85	2008-04-08	11
-86	val_86	2008-04-08	11
-87	val_87	2008-04-08	11
-90	val_90	2008-04-08	11
-90	val_90	2008-04-08	11
-90	val_90	2008-04-08	11
-92	val_92	2008-04-08	11
-95	val_95	2008-04-08	11
-95	val_95	2008-04-08	11
-96	val_96	2008-04-08	11
-97	val_97	2008-04-08	11
-97	val_97	2008-04-08	11
-98	val_98	2008-04-08	11
-98	val_98	2008-04-08	11
-100	val_100	2008-04-08	11
-100	val_100	2008-04-08	11
-103	val_103	2008-04-08	11
-103	val_103	2008-04-08	11
-104	val_104	2008-04-08	11
-104	val_104	2008-04-08	11
-105	val_105	2008-04-08	11
-111	val_111	2008-04-08	11
-113	val_113	2008-04-08	11
-113	val_113	2008-04-08	11
-114	val_114	2008-04-08	11
-116	val_116	2008-04-08	11
-118	val_118	2008-04-08	11
-118	val_118	2008-04-08	11
-119	val_119	2008-04-08	11
-119	val_119	2008-04-08	11
-119	val_119	2008-04-08	11
-120	val_120	2008-04-08	11
-120	val_120	2008-04-08	11
-125	val_125	2008-04-08	11
-125	val_125	2008-04-08	11
-126	val_126	2008-04-08	11
-128	val_128	2008-04-08	11
-128	val_128	2008-04-08	11
-128	val_128	2008-04-08	11
-129	val_129	2008-04-08	11
-129	val_129	2008-04-08	11
-131	val_131	2008-04-08	11
-133	val_133	2008-04-08	11
-134	val_134	2008-04-08	11
-134	val_134	2008-04-08	11
-136	val_136	2008-04-08	11
-137	val_137	2008-04-08	11
-137	val_137	2008-04-08	11
-138	val_138	2008-04-08	11
-138	val_138	2008-04-08	11
-138	val_138	2008-04-08	11
-138	val_138	2008-04-08	11
-143	val_143	2008-04-08	11
-145	val_145	2008-04-08	11
-146	val_146	2008-04-08	11
-146	val_146	2008-04-08	11
-149	val_149	2008-04-08	11
-149	val_149	2008-04-08	11
-150	val_150	2008-04-08	11
-152	val_152	2008-04-08	11
-152	val_152	2008-04-08	11
-153	val_153	2008-04-08	11
-155	val_155	2008-04-08	11
-156	val_156	2008-04-08	11
-157	val_157	2008-04-08	11
-158	val_158	2008-04-08	11
-160	val_160	2008-04-08	11
-162	val_162	2008-04-08	11
-163	val_163	2008-04-08	11
-164	val_164	2008-04-08	11
-164	val_164	2008-04-08	11
-165	val_165	2008-04-08	11
-165	val_165	2008-04-08	11
-166	val_166	2008-04-08	11
-167	val_167	2008-04-08	11
-167	val_167	2008-04-08	11
-167	val_167	2008-04-08	11
-168	val_168	2008-04-08	11
-169	val_169	2008-04-08	11
-169	val_169	2008-04-08	11
-169	val_169	2008-04-08	11
-169	val_169	2008-04-08	11
-170	val_170	2008-04-08	11
-172	val_172	2008-04-08	11
-172	val_172	2008-04-08	11
-174	val_174	2008-04-08	11
-174	val_174	2008-04-08	11
-175	val_175	2008-04-08	11
-175	val_175	2008-04-08	11
-176	val_176	2008-04-08	11
-176	val_176	2008-04-08	11
-177	val_177	2008-04-08	11
-178	val_178	2008-04-08	11
-179	val_179	2008-04-08	11
-179	val_179	2008-04-08	11
-180	val_180	2008-04-08	11
-181	val_181	2008-04-08	11
-183	val_183	2008-04-08	11
-186	val_186	2008-04-08	11
-187	val_187	2008-04-08	11
-187	val_187	2008-04-08	11
-187	val_187	2008-04-08	11
-189	val_189	2008-04-08	11
-190	val_190	2008-04-08	11
-191	val_191	2008-04-08	11
-191	val_191	2008-04-08	11
-192	val_192	2008-04-08	11
-193	val_193	2008-04-08	11
-193	val_193	2008-04-08	11
-193	val_193	2008-04-08	11
-194	val_194	2008-04-08	11
-195	val_195	2008-04-08	11
-195	val_195	2008-04-08	11
-196	val_196	2008-04-08	11
-197	val_197	2008-04-08	11
-197	val_197	2008-04-08	11
-199	val_199	2008-04-08	11
-199	val_199	2008-04-08	11
-199	val_199	2008-04-08	11
-200	val_200	2008-04-08	11
-200	val_200	2008-04-08	11
-201	val_201	2008-04-08	11
-202	val_202	2008-04-08	11
-203	val_203	2008-04-08	11
-203	val_203	2008-04-08	11
-205	val_205	2008-04-08	11
-205	val_205	2008-04-08	11
-207	val_207	2008-04-08	11
-207	val_207	2008-04-08	11
-208	val_208	2008-04-08	11
-208	val_208	2008-04-08	11
-208	val_208	2008-04-08	11
-209	val_209	2008-04-08	11
-209	val_209	2008-04-08	11
-213	val_213	2008-04-08	11
-213	val_213	2008-04-08	11
-214	val_214	2008-04-08	11
-216	val_216	2008-04-08	11
-216	val_216	2008-04-08	11
-217	val_217	2008-04-08	11
-217	val_217	2008-04-08	11
-218	val_218	2008-04-08	11
-219	val_219	2008-04-08	11
-219	val_219	2008-04-08	11
-221	val_221	2008-04-08	11
-221	val_221	2008-04-08	11
-222	val_222	2008-04-08	11
-223	val_223	2008-04-08	11
-223	val_223	2008-04-08	11
-224	val_224	2008-04-08	11
-224	val_224	2008-04-08	11
-226	val_226	2008-04-08	11
-228	val_228	2008-04-08	11
-229	val_229	2008-04-08	11
-229	val_229	2008-04-08	11
-230	val_230	2008-04-08	11
-230	val_230	2008-04-08	11
-230	val_230	2008-04-08	11
-230	val_230	2008-04-08	11
-230	val_230	2008-04-08	11
-233	val_233	2008-04-08	11
-233	val_233	2008-04-08	11
-235	val_235	2008-04-08	11
-237	val_237	2008-04-08	11
-237	val_237	2008-04-08	11
-238	val_238	2008-04-08	11
-238	val_238	2008-04-08	11
-239	val_239	2008-04-08	11
-239	val_239	2008-04-08	11
-241	val_241	2008-04-08	11
-242	val_242	2008-04-08	11
-242	val_242	2008-04-08	11
-244	val_244	2008-04-08	11
-247	val_247	2008-04-08	11
-248	val_248	2008-04-08	11
-249	val_249	2008-04-08	11
-252	val_252	2008-04-08	11
-255	val_255	2008-04-08	11
-255	val_255	2008-04-08	11
-256	val_256	2008-04-08	11
-256	val_256	2008-04-08	11
-257	val_257	2008-04-08	11
-258	val_258	2008-04-08	11
-260	val_260	2008-04-08	11
-262	val_262	2008-04-08	11
-263	val_263	2008-04-08	11
-265	val_265	2008-04-08	11
-265	val_265	2008-04-08	11
-266	val_266	2008-04-08	11
-272	val_272	2008-04-08	11
-272	val_272	2008-04-08	11
-273	val_273	2008-04-08	11
-273	val_273	2008-04-08	11
-273	val_273	2008-04-08	11
-274	val_274	2008-04-08	11
-275	val_275	2008-04-08	11
-277	val_277	2008-04-08	11
-277	val_277	2008-04-08	11
-277	val_277	2008-04-08	11
-277	val_277	2008-04-08	11
-278	val_278	2008-04-08	11
-278	val_278	2008-04-08	11
-280	val_280	2008-04-08	11
-280	val_280	2008-04-08	11
-281	val_281	2008-04-08	11
-281	val_281	2008-04-08	11
-282	val_282	2008-04-08	11
-282	val_282	2008-04-08	11
-283	val_283	2008-04-08	11
-284	val_284	2008-04-08	11
-285	val_285	2008-04-08	11
-286	val_286	2008-04-08	11
-287	val_287	2008-04-08	11
-288	val_288	2008-04-08	11
-288	val_288	2008-04-08	11
-289	val_289	2008-04-08	11
-291	val_291	2008-04-08	11
-292	val_292	2008-04-08	11
-296	val_296	2008-04-08	11
-298	val_298	2008-04-08	11
-298	val_298	2008-04-08	11
-298	val_298	2008-04-08	11
-302	val_302	2008-04-08	11
-305	val_305	2008-04-08	11
-306	val_306	2008-04-08	11
-307	val_307	2008-04-08	11
-307	val_307	2008-04-08	11
-308	val_308	2008-04-08	11
-309	val_309	2008-04-08	11
-309	val_309	2008-04-08	11
-310	val_310	2008-04-08	11
-311	val_311	2008-04-08	11
-311	val_311	2008-04-08	11
-311	val_311	2008-04-08	11
-315	val_315	2008-04-08	11
-316	val_316	2008-04-08	11
-316	val_316	2008-04-08	11
-316	val_316	2008-04-08	11
-317	val_317	2008-04-08	11
-317	val_317	2008-04-08	11
-318	val_318	2008-04-08	11
-318	val_318	2008-04-08	11
-318	val_318	2008-04-08	11
-321	val_321	2008-04-08	11
-321	val_321	2008-04-08	11
-322	val_322	2008-04-08	11
-322	val_322	2008-04-08	11
-323	val_323	2008-04-08	11
-325	val_325	2008-04-08	11
-325	val_325	2008-04-08	11
-327	val_327	2008-04-08	11
-327	val_327	2008-04-08	11
-327	val_327	2008-04-08	11
-331	val_331	2008-04-08	11
-331	val_331	2008-04-08	11
-332	val_332	2008-04-08	11
-333	val_333	2008-04-08	11
-333	val_333	2008-04-08	11
-335	val_335	2008-04-08	11
-336	val_336	2008-04-08	11
-338	val_338	2008-04-08	11
-339	val_339	2008-04-08	11
-341	val_341	2008-04-08	11
-342	val_342	2008-04-08	11
-342	val_342	2008-04-08	11
-344	val_344	2008-04-08	11
-344	val_344	2008-04-08	11
-345	val_345	2008-04-08	11
-348	val_348	2008-04-08	11
-348	val_348	2008-04-08	11
-348	val_348	2008-04-08	11
-348	val_348	2008-04-08	11
-348	val_348	2008-04-08	11
-351	val_351	2008-04-08	11
-353	val_353	2008-04-08	11
-353	val_353	2008-04-08	11
-356	val_356	2008-04-08	11
-360	val_360	2008-04-08	11
-362	val_362	2008-04-08	11
-364	val_364	2008-04-08	11
-365	val_365	2008-04-08	11
-366	val_366	2008-04-08	11
-367	val_367	2008-04-08	11
-367	val_367	2008-04-08	11
-368	val_368	2008-04-08	11
-369	val_369	2008-04-08	11
-369	val_369	2008-04-08	11
-369	val_369	2008-04-08	11
-373	val_373	2008-04-08	11
-374	val_374	2008-04-08	11
-375	val_375	2008-04-08	11
-377	val_377	2008-04-08	11
-378	val_378	2008-04-08	11
-379	val_379	2008-04-08	11
-382	val_382	2008-04-08	11
-382	val_382	2008-04-08	11
-384	val_384	2008-04-08	11
-384	val_384	2008-04-08	11
-384	val_384	2008-04-08	11
-386	val_386	2008-04-08	11
-389	val_389	2008-04-08	11
-392	val_392	2008-04-08	11
-393	val_393	2008-04-08	11
-394	val_394	2008-04-08	11
-395	val_395	2008-04-08	11
-395	val_395	2008-04-08	11
-396	val_396	2008-04-08	11
-396	val_396	2008-04-08	11
-396	val_396	2008-04-08	11
-397	val_397	2008-04-08	11
-397	val_397	2008-04-08	11
-399	val_399	2008-04-08	11
-399	val_399	2008-04-08	11
-400	val_400	2008-04-08	11
-401	val_401	2008-04-08	11
-401	val_401	2008-04-08	11
-401	val_401	2008-04-08	11
-401	val_401	2008-04-08	11
-401	val_401	2008-04-08	11
-402	val_402	2008-04-08	11
-403	val_403	2008-04-08	11
-403	val_403	2008-04-08	11
-403	val_403	2008-04-08	11
-404	val_404	2008-04-08	11
-404	val_404	2008-04-08	11
-406	val_406	2008-04-08	11
-406	val_406	2008-04-08	11
-406	val_406	2008-04-08	11
-406	val_406	2008-04-08	11
-407	val_407	2008-04-08	11
-409	val_409	2008-04-08	11
-409	val_409	2008-04-08	11
-409	val_409	2008-04-08	11
-411	val_411	2008-04-08	11
-413	val_413	2008-04-08	11
-413	val_413	2008-04-08	11
-414	val_414	2008-04-08	11
-414	val_414	2008-04-08	11
-417	val_417	2008-04-08	11
-417	val_417	2008-04-08	11
-417	val_417	2008-04-08	11
-418	val_418	2008-04-08	11
-419	val_419	2008-04-08	11
-421	val_421	2008-04-08	11
-424	val_424	2008-04-08	11
-424	val_424	2008-04-08	11
-427	val_427	2008-04-08	11
-429	val_429	2008-04-08	11
-429	val_429	2008-04-08	11
-430	val_430	2008-04-08	11
-430	val_430	2008-04-08	11
-430	val_430	2008-04-08	11
-431	val_431	2008-04-08	11
-431	val_431	2008-04-08	11
-431	val_431	2008-04-08	11
-432	val_432	2008-04-08	11
-435	val_435	2008-04-08	11
-436	val_436	2008-04-08	11
-437	val_437	2008-04-08	11
-438	val_438	2008-04-08	11
-438	val_438	2008-04-08	11
-438	val_438	2008-04-08	11
-439	val_439	2008-04-08	11
-439	val_439	2008-04-08	11
-443	val_443	2008-04-08	11
-444	val_444	2008-04-08	11
-446	val_446	2008-04-08	11
-448	val_448	2008-04-08	11
-449	val_449	2008-04-08	11
-452	val_452	2008-04-08	11
-453	val_453	2008-04-08	11
-454	val_454	2008-04-08	11
-454	val_454	2008-04-08	11
-454	val_454	2008-04-08	11
-455	val_455	2008-04-08	11
-457	val_457	2008-04-08	11
-458	val_458	2008-04-08	11
-458	val_458	2008-04-08	11
-459	val_459	2008-04-08	11
-459	val_459	2008-04-08	11
-460	val_460	2008-04-08	11
-462	val_462	2008-04-08	11
-462	val_462	2008-04-08	11
-463	val_463	2008-04-08	11
-463	val_463	2008-04-08	11
-466	val_466	2008-04-08	11
-466	val_466	2008-04-08	11
-466	val_466	2008-04-08	11
-467	val_467	2008-04-08	11
-468	val_468	2008-04-08	11
-468	val_468	2008-04-08	11
-468	val_468	2008-04-08	11
-468	val_468	2008-04-08	11
-469	val_469	2008-04-08	11
-469	val_469	2008-04-08	11
-469	val_469	2008-04-08	11
-469	val_469	2008-04-08	11
-469	val_469	2008-04-08	11
-470	val_470	2008-04-08	11
-472	val_472	2008-04-08	11
-475	val_475	2008-04-08	11
-477	val_477	2008-04-08	11
-478	val_478	2008-04-08	11
-478	val_478	2008-04-08	11
-479	val_479	2008-04-08	11
-480	val_480	2008-04-08	11
-480	val_480	2008-04-08	11
-480	val_480	2008-04-08	11
-481	val_481	2008-04-08	11
-482	val_482	2008-04-08	11
-483	val_483	2008-04-08	11
-484	val_484	2008-04-08	11
-485	val_485	2008-04-08	11
-487	val_487	2008-04-08	11
-489	val_489	2008-04-08	11
-489	val_489	2008-04-08	11
-489	val_489	2008-04-08	11
-489	val_489	2008-04-08	11
-490	val_490	2008-04-08	11
-491	val_491	2008-04-08	11
-492	val_492	2008-04-08	11
-492	val_492	2008-04-08	11
-493	val_493	2008-04-08	11
-494	val_494	2008-04-08	11
-495	val_495	2008-04-08	11
-496	val_496	2008-04-08	11
-497	val_497	2008-04-08	11
-498	val_498	2008-04-08	11
-498	val_498	2008-04-08	11
-498	val_498	2008-04-08	11
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858
deleted file mode 100644
index 352ab5a6b96a4f2945ba7f3061a877d95abcb94a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:merge_dynamic_part
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1431818582215388621/merge_dynamic_part
-inputformat:org.apache.hadoop.mapred.TextInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-columns:struct columns { i32 key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string ds, string hr}
-totalNumberFiles:1
-totalFileSize:5812
-maxFileSize:5812
-minFileSize:5812
-lastAccessTime:0
-lastUpdateTime:1389738939000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf b/sql/hive/src/test/resources/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b b/sql/hive/src/test/resources/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858
deleted file mode 100644
index f3812861b3d6d312f31b29d66dbb6433d60ddd6a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:merge_dynamic_part
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5646492510204438812/merge_dynamic_part
-inputformat:org.apache.hadoop.mapred.TextInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-columns:struct columns { i32 key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string ds, string hr}
-totalNumberFiles:3
-totalFileSize:17415
-maxFileSize:5901
-minFileSize:5702
-lastAccessTime:0
-lastUpdateTime:1389740265000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73
deleted file mode 100644
index e9c723bbd136e561e0ff75d4b248e96ccbf81e73..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73
+++ /dev/null
@@ -1,4 +0,0 @@
-ds=2008-04-08/hr=11
-ds=2008-04-08/hr=12
-ds=2008-04-09/hr=11
-ds=2008-04-09/hr=12
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a
deleted file mode 100644
index 5e44ab6b5cef921a7c303429c98fdfca879df372..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a
+++ /dev/null
@@ -1,4 +0,0 @@
-2008-04-08	11	500
-2008-04-08	12	500
-2008-04-09	11	1000
-2008-04-09	12	1000
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858
deleted file mode 100644
index 8cd9e4d2c201ab1129c606ca2bd5a5ae04033c86..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:merge_dynamic_part
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/merge_dynamic_part
-inputformat:org.apache.hadoop.mapred.TextInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-columns:struct columns { i32 key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string ds, string hr}
-totalNumberFiles:6
-totalFileSize:34830
-maxFileSize:5812
-minFileSize:5791
-lastAccessTime:0
-lastUpdateTime:1389739573000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 b/sql/hive/src/test/resources/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 b/sql/hive/src/test/resources/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 b/sql/hive/src/test/resources/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 b/sql/hive/src/test/resources/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b b/sql/hive/src/test/resources/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff b/sql/hive/src/test/resources/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 b/sql/hive/src/test/resources/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 b/sql/hive/src/test/resources/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 b/sql/hive/src/test/resources/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491 b/sql/hive/src/test/resources/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 b/sql/hive/src/test/resources/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 b/sql/hive/src/test/resources/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 b/sql/hive/src/test/resources/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d b/sql/hive/src/test/resources/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 b/sql/hive/src/test/resources/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d b/sql/hive/src/test/resources/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 b/sql/hive/src/test/resources/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67 b/sql/hive/src/test/resources/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 b/sql/hive/src/test/resources/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf b/sql/hive/src/test/resources/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 b/sql/hive/src/test/resources/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/mi-0-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/mi-0-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/mi-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/mi-1-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/mi-1-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e b/sql/hive/src/test/resources/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mi-3-b66a495f7bdf106a7886b72267b8659d b/sql/hive/src/test/resources/golden/mi-3-b66a495f7bdf106a7886b72267b8659d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c b/sql/hive/src/test/resources/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc b/sql/hive/src/test/resources/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc
deleted file mode 100644
index 3b011a048ba429624b26c8f3383f19b5030cb516..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc
+++ /dev/null
@@ -1 +0,0 @@
-ds=2008-04-08/hr=11
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 b/sql/hive/src/test/resources/golden/mi-6-b331d7aa963adac3b2e1de803f90e199
deleted file mode 100644
index 0cdd3e8594c5991436a85d167f25fd33fc1198f2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/mi-6-b331d7aa963adac3b2e1de803f90e199
+++ /dev/null
@@ -1 +0,0 @@
-ds=2008-04-08/hr=12
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 b/sql/hive/src/test/resources/golden/mi-7-fca4e06de103c3cbb675fa43e7077800
deleted file mode 100644
index f4026a591a958fbe6b680ee771bfa632a2fc0d72..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/mi-7-fca4e06de103c3cbb675fa43e7077800
+++ /dev/null
@@ -1,500 +0,0 @@
-238	val_238	2008-04-08	11
-86	val_86	2008-04-08	11
-311	val_311	2008-04-08	11
-27	val_27	2008-04-08	11
-165	val_165	2008-04-08	11
-409	val_409	2008-04-08	11
-255	val_255	2008-04-08	11
-278	val_278	2008-04-08	11
-98	val_98	2008-04-08	11
-484	val_484	2008-04-08	11
-265	val_265	2008-04-08	11
-193	val_193	2008-04-08	11
-401	val_401	2008-04-08	11
-150	val_150	2008-04-08	11
-273	val_273	2008-04-08	11
-224	val_224	2008-04-08	11
-369	val_369	2008-04-08	11
-66	val_66	2008-04-08	11
-128	val_128	2008-04-08	11
-213	val_213	2008-04-08	11
-146	val_146	2008-04-08	11
-406	val_406	2008-04-08	11
-429	val_429	2008-04-08	11
-374	val_374	2008-04-08	11
-152	val_152	2008-04-08	11
-469	val_469	2008-04-08	11
-145	val_145	2008-04-08	11
-495	val_495	2008-04-08	11
-37	val_37	2008-04-08	11
-327	val_327	2008-04-08	11
-281	val_281	2008-04-08	11
-277	val_277	2008-04-08	11
-209	val_209	2008-04-08	11
-15	val_15	2008-04-08	11
-82	val_82	2008-04-08	11
-403	val_403	2008-04-08	11
-166	val_166	2008-04-08	11
-417	val_417	2008-04-08	11
-430	val_430	2008-04-08	11
-252	val_252	2008-04-08	11
-292	val_292	2008-04-08	11
-219	val_219	2008-04-08	11
-287	val_287	2008-04-08	11
-153	val_153	2008-04-08	11
-193	val_193	2008-04-08	11
-338	val_338	2008-04-08	11
-446	val_446	2008-04-08	11
-459	val_459	2008-04-08	11
-394	val_394	2008-04-08	11
-237	val_237	2008-04-08	11
-482	val_482	2008-04-08	11
-174	val_174	2008-04-08	11
-413	val_413	2008-04-08	11
-494	val_494	2008-04-08	11
-207	val_207	2008-04-08	11
-199	val_199	2008-04-08	11
-466	val_466	2008-04-08	11
-208	val_208	2008-04-08	11
-174	val_174	2008-04-08	11
-399	val_399	2008-04-08	11
-396	val_396	2008-04-08	11
-247	val_247	2008-04-08	11
-417	val_417	2008-04-08	11
-489	val_489	2008-04-08	11
-162	val_162	2008-04-08	11
-377	val_377	2008-04-08	11
-397	val_397	2008-04-08	11
-309	val_309	2008-04-08	11
-365	val_365	2008-04-08	11
-266	val_266	2008-04-08	11
-439	val_439	2008-04-08	11
-342	val_342	2008-04-08	11
-367	val_367	2008-04-08	11
-325	val_325	2008-04-08	11
-167	val_167	2008-04-08	11
-195	val_195	2008-04-08	11
-475	val_475	2008-04-08	11
-17	val_17	2008-04-08	11
-113	val_113	2008-04-08	11
-155	val_155	2008-04-08	11
-203	val_203	2008-04-08	11
-339	val_339	2008-04-08	11
-0	val_0	2008-04-08	11
-455	val_455	2008-04-08	11
-128	val_128	2008-04-08	11
-311	val_311	2008-04-08	11
-316	val_316	2008-04-08	11
-57	val_57	2008-04-08	11
-302	val_302	2008-04-08	11
-205	val_205	2008-04-08	11
-149	val_149	2008-04-08	11
-438	val_438	2008-04-08	11
-345	val_345	2008-04-08	11
-129	val_129	2008-04-08	11
-170	val_170	2008-04-08	11
-20	val_20	2008-04-08	11
-489	val_489	2008-04-08	11
-157	val_157	2008-04-08	11
-378	val_378	2008-04-08	11
-221	val_221	2008-04-08	11
-92	val_92	2008-04-08	11
-111	val_111	2008-04-08	11
-47	val_47	2008-04-08	11
-72	val_72	2008-04-08	11
-4	val_4	2008-04-08	11
-280	val_280	2008-04-08	11
-35	val_35	2008-04-08	11
-427	val_427	2008-04-08	11
-277	val_277	2008-04-08	11
-208	val_208	2008-04-08	11
-356	val_356	2008-04-08	11
-399	val_399	2008-04-08	11
-169	val_169	2008-04-08	11
-382	val_382	2008-04-08	11
-498	val_498	2008-04-08	11
-125	val_125	2008-04-08	11
-386	val_386	2008-04-08	11
-437	val_437	2008-04-08	11
-469	val_469	2008-04-08	11
-192	val_192	2008-04-08	11
-286	val_286	2008-04-08	11
-187	val_187	2008-04-08	11
-176	val_176	2008-04-08	11
-54	val_54	2008-04-08	11
-459	val_459	2008-04-08	11
-51	val_51	2008-04-08	11
-138	val_138	2008-04-08	11
-103	val_103	2008-04-08	11
-239	val_239	2008-04-08	11
-213	val_213	2008-04-08	11
-216	val_216	2008-04-08	11
-430	val_430	2008-04-08	11
-278	val_278	2008-04-08	11
-176	val_176	2008-04-08	11
-289	val_289	2008-04-08	11
-221	val_221	2008-04-08	11
-65	val_65	2008-04-08	11
-318	val_318	2008-04-08	11
-332	val_332	2008-04-08	11
-311	val_311	2008-04-08	11
-275	val_275	2008-04-08	11
-137	val_137	2008-04-08	11
-241	val_241	2008-04-08	11
-83	val_83	2008-04-08	11
-333	val_333	2008-04-08	11
-180	val_180	2008-04-08	11
-284	val_284	2008-04-08	11
-12	val_12	2008-04-08	11
-230	val_230	2008-04-08	11
-181	val_181	2008-04-08	11
-67	val_67	2008-04-08	11
-260	val_260	2008-04-08	11
-404	val_404	2008-04-08	11
-384	val_384	2008-04-08	11
-489	val_489	2008-04-08	11
-353	val_353	2008-04-08	11
-373	val_373	2008-04-08	11
-272	val_272	2008-04-08	11
-138	val_138	2008-04-08	11
-217	val_217	2008-04-08	11
-84	val_84	2008-04-08	11
-348	val_348	2008-04-08	11
-466	val_466	2008-04-08	11
-58	val_58	2008-04-08	11
-8	val_8	2008-04-08	11
-411	val_411	2008-04-08	11
-230	val_230	2008-04-08	11
-208	val_208	2008-04-08	11
-348	val_348	2008-04-08	11
-24	val_24	2008-04-08	11
-463	val_463	2008-04-08	11
-431	val_431	2008-04-08	11
-179	val_179	2008-04-08	11
-172	val_172	2008-04-08	11
-42	val_42	2008-04-08	11
-129	val_129	2008-04-08	11
-158	val_158	2008-04-08	11
-119	val_119	2008-04-08	11
-496	val_496	2008-04-08	11
-0	val_0	2008-04-08	11
-322	val_322	2008-04-08	11
-197	val_197	2008-04-08	11
-468	val_468	2008-04-08	11
-393	val_393	2008-04-08	11
-454	val_454	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-199	val_199	2008-04-08	11
-191	val_191	2008-04-08	11
-418	val_418	2008-04-08	11
-96	val_96	2008-04-08	11
-26	val_26	2008-04-08	11
-165	val_165	2008-04-08	11
-327	val_327	2008-04-08	11
-230	val_230	2008-04-08	11
-205	val_205	2008-04-08	11
-120	val_120	2008-04-08	11
-131	val_131	2008-04-08	11
-51	val_51	2008-04-08	11
-404	val_404	2008-04-08	11
-43	val_43	2008-04-08	11
-436	val_436	2008-04-08	11
-156	val_156	2008-04-08	11
-469	val_469	2008-04-08	11
-468	val_468	2008-04-08	11
-308	val_308	2008-04-08	11
-95	val_95	2008-04-08	11
-196	val_196	2008-04-08	11
-288	val_288	2008-04-08	11
-481	val_481	2008-04-08	11
-457	val_457	2008-04-08	11
-98	val_98	2008-04-08	11
-282	val_282	2008-04-08	11
-197	val_197	2008-04-08	11
-187	val_187	2008-04-08	11
-318	val_318	2008-04-08	11
-318	val_318	2008-04-08	11
-409	val_409	2008-04-08	11
-470	val_470	2008-04-08	11
-137	val_137	2008-04-08	11
-369	val_369	2008-04-08	11
-316	val_316	2008-04-08	11
-169	val_169	2008-04-08	11
-413	val_413	2008-04-08	11
-85	val_85	2008-04-08	11
-77	val_77	2008-04-08	11
-0	val_0	2008-04-08	11
-490	val_490	2008-04-08	11
-87	val_87	2008-04-08	11
-364	val_364	2008-04-08	11
-179	val_179	2008-04-08	11
-118	val_118	2008-04-08	11
-134	val_134	2008-04-08	11
-395	val_395	2008-04-08	11
-282	val_282	2008-04-08	11
-138	val_138	2008-04-08	11
-238	val_238	2008-04-08	11
-419	val_419	2008-04-08	11
-15	val_15	2008-04-08	11
-118	val_118	2008-04-08	11
-72	val_72	2008-04-08	11
-90	val_90	2008-04-08	11
-307	val_307	2008-04-08	11
-19	val_19	2008-04-08	11
-435	val_435	2008-04-08	11
-10	val_10	2008-04-08	11
-277	val_277	2008-04-08	11
-273	val_273	2008-04-08	11
-306	val_306	2008-04-08	11
-224	val_224	2008-04-08	11
-309	val_309	2008-04-08	11
-389	val_389	2008-04-08	11
-327	val_327	2008-04-08	11
-242	val_242	2008-04-08	11
-369	val_369	2008-04-08	11
-392	val_392	2008-04-08	11
-272	val_272	2008-04-08	11
-331	val_331	2008-04-08	11
-401	val_401	2008-04-08	11
-242	val_242	2008-04-08	11
-452	val_452	2008-04-08	11
-177	val_177	2008-04-08	11
-226	val_226	2008-04-08	11
-5	val_5	2008-04-08	11
-497	val_497	2008-04-08	11
-402	val_402	2008-04-08	11
-396	val_396	2008-04-08	11
-317	val_317	2008-04-08	11
-395	val_395	2008-04-08	11
-58	val_58	2008-04-08	11
-35	val_35	2008-04-08	11
-336	val_336	2008-04-08	11
-95	val_95	2008-04-08	11
-11	val_11	2008-04-08	11
-168	val_168	2008-04-08	11
-34	val_34	2008-04-08	11
-229	val_229	2008-04-08	11
-233	val_233	2008-04-08	11
-143	val_143	2008-04-08	11
-472	val_472	2008-04-08	11
-322	val_322	2008-04-08	11
-498	val_498	2008-04-08	11
-160	val_160	2008-04-08	11
-195	val_195	2008-04-08	11
-42	val_42	2008-04-08	11
-321	val_321	2008-04-08	11
-430	val_430	2008-04-08	11
-119	val_119	2008-04-08	11
-489	val_489	2008-04-08	11
-458	val_458	2008-04-08	11
-78	val_78	2008-04-08	11
-76	val_76	2008-04-08	11
-41	val_41	2008-04-08	11
-223	val_223	2008-04-08	11
-492	val_492	2008-04-08	11
-149	val_149	2008-04-08	11
-449	val_449	2008-04-08	11
-218	val_218	2008-04-08	11
-228	val_228	2008-04-08	11
-138	val_138	2008-04-08	11
-453	val_453	2008-04-08	11
-30	val_30	2008-04-08	11
-209	val_209	2008-04-08	11
-64	val_64	2008-04-08	11
-468	val_468	2008-04-08	11
-76	val_76	2008-04-08	11
-74	val_74	2008-04-08	11
-342	val_342	2008-04-08	11
-69	val_69	2008-04-08	11
-230	val_230	2008-04-08	11
-33	val_33	2008-04-08	11
-368	val_368	2008-04-08	11
-103	val_103	2008-04-08	11
-296	val_296	2008-04-08	11
-113	val_113	2008-04-08	11
-216	val_216	2008-04-08	11
-367	val_367	2008-04-08	11
-344	val_344	2008-04-08	11
-167	val_167	2008-04-08	11
-274	val_274	2008-04-08	11
-219	val_219	2008-04-08	11
-239	val_239	2008-04-08	11
-485	val_485	2008-04-08	11
-116	val_116	2008-04-08	11
-223	val_223	2008-04-08	11
-256	val_256	2008-04-08	11
-263	val_263	2008-04-08	11
-70	val_70	2008-04-08	11
-487	val_487	2008-04-08	11
-480	val_480	2008-04-08	11
-401	val_401	2008-04-08	11
-288	val_288	2008-04-08	11
-191	val_191	2008-04-08	11
-5	val_5	2008-04-08	11
-244	val_244	2008-04-08	11
-438	val_438	2008-04-08	11
-128	val_128	2008-04-08	11
-467	val_467	2008-04-08	11
-432	val_432	2008-04-08	11
-202	val_202	2008-04-08	11
-316	val_316	2008-04-08	11
-229	val_229	2008-04-08	11
-469	val_469	2008-04-08	11
-463	val_463	2008-04-08	11
-280	val_280	2008-04-08	11
-2	val_2	2008-04-08	11
-35	val_35	2008-04-08	11
-283	val_283	2008-04-08	11
-331	val_331	2008-04-08	11
-235	val_235	2008-04-08	11
-80	val_80	2008-04-08	11
-44	val_44	2008-04-08	11
-193	val_193	2008-04-08	11
-321	val_321	2008-04-08	11
-335	val_335	2008-04-08	11
-104	val_104	2008-04-08	11
-466	val_466	2008-04-08	11
-366	val_366	2008-04-08	11
-175	val_175	2008-04-08	11
-403	val_403	2008-04-08	11
-483	val_483	2008-04-08	11
-53	val_53	2008-04-08	11
-105	val_105	2008-04-08	11
-257	val_257	2008-04-08	11
-406	val_406	2008-04-08	11
-409	val_409	2008-04-08	11
-190	val_190	2008-04-08	11
-406	val_406	2008-04-08	11
-401	val_401	2008-04-08	11
-114	val_114	2008-04-08	11
-258	val_258	2008-04-08	11
-90	val_90	2008-04-08	11
-203	val_203	2008-04-08	11
-262	val_262	2008-04-08	11
-348	val_348	2008-04-08	11
-424	val_424	2008-04-08	11
-12	val_12	2008-04-08	11
-396	val_396	2008-04-08	11
-201	val_201	2008-04-08	11
-217	val_217	2008-04-08	11
-164	val_164	2008-04-08	11
-431	val_431	2008-04-08	11
-454	val_454	2008-04-08	11
-478	val_478	2008-04-08	11
-298	val_298	2008-04-08	11
-125	val_125	2008-04-08	11
-431	val_431	2008-04-08	11
-164	val_164	2008-04-08	11
-424	val_424	2008-04-08	11
-187	val_187	2008-04-08	11
-382	val_382	2008-04-08	11
-5	val_5	2008-04-08	11
-70	val_70	2008-04-08	11
-397	val_397	2008-04-08	11
-480	val_480	2008-04-08	11
-291	val_291	2008-04-08	11
-24	val_24	2008-04-08	11
-351	val_351	2008-04-08	11
-255	val_255	2008-04-08	11
-104	val_104	2008-04-08	11
-70	val_70	2008-04-08	11
-163	val_163	2008-04-08	11
-438	val_438	2008-04-08	11
-119	val_119	2008-04-08	11
-414	val_414	2008-04-08	11
-200	val_200	2008-04-08	11
-491	val_491	2008-04-08	11
-237	val_237	2008-04-08	11
-439	val_439	2008-04-08	11
-360	val_360	2008-04-08	11
-248	val_248	2008-04-08	11
-479	val_479	2008-04-08	11
-305	val_305	2008-04-08	11
-417	val_417	2008-04-08	11
-199	val_199	2008-04-08	11
-444	val_444	2008-04-08	11
-120	val_120	2008-04-08	11
-429	val_429	2008-04-08	11
-169	val_169	2008-04-08	11
-443	val_443	2008-04-08	11
-323	val_323	2008-04-08	11
-325	val_325	2008-04-08	11
-277	val_277	2008-04-08	11
-230	val_230	2008-04-08	11
-478	val_478	2008-04-08	11
-178	val_178	2008-04-08	11
-468	val_468	2008-04-08	11
-310	val_310	2008-04-08	11
-317	val_317	2008-04-08	11
-333	val_333	2008-04-08	11
-493	val_493	2008-04-08	11
-460	val_460	2008-04-08	11
-207	val_207	2008-04-08	11
-249	val_249	2008-04-08	11
-265	val_265	2008-04-08	11
-480	val_480	2008-04-08	11
-83	val_83	2008-04-08	11
-136	val_136	2008-04-08	11
-353	val_353	2008-04-08	11
-172	val_172	2008-04-08	11
-214	val_214	2008-04-08	11
-462	val_462	2008-04-08	11
-233	val_233	2008-04-08	11
-406	val_406	2008-04-08	11
-133	val_133	2008-04-08	11
-175	val_175	2008-04-08	11
-189	val_189	2008-04-08	11
-454	val_454	2008-04-08	11
-375	val_375	2008-04-08	11
-401	val_401	2008-04-08	11
-421	val_421	2008-04-08	11
-407	val_407	2008-04-08	11
-384	val_384	2008-04-08	11
-256	val_256	2008-04-08	11
-26	val_26	2008-04-08	11
-134	val_134	2008-04-08	11
-67	val_67	2008-04-08	11
-384	val_384	2008-04-08	11
-379	val_379	2008-04-08	11
-18	val_18	2008-04-08	11
-462	val_462	2008-04-08	11
-492	val_492	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-9	val_9	2008-04-08	11
-341	val_341	2008-04-08	11
-498	val_498	2008-04-08	11
-146	val_146	2008-04-08	11
-458	val_458	2008-04-08	11
-362	val_362	2008-04-08	11
-186	val_186	2008-04-08	11
-285	val_285	2008-04-08	11
-348	val_348	2008-04-08	11
-167	val_167	2008-04-08	11
-18	val_18	2008-04-08	11
-273	val_273	2008-04-08	11
-183	val_183	2008-04-08	11
-281	val_281	2008-04-08	11
-344	val_344	2008-04-08	11
-97	val_97	2008-04-08	11
-469	val_469	2008-04-08	11
-315	val_315	2008-04-08	11
-84	val_84	2008-04-08	11
-28	val_28	2008-04-08	11
-37	val_37	2008-04-08	11
-448	val_448	2008-04-08	11
-152	val_152	2008-04-08	11
-348	val_348	2008-04-08	11
-307	val_307	2008-04-08	11
-194	val_194	2008-04-08	11
-414	val_414	2008-04-08	11
-477	val_477	2008-04-08	11
-222	val_222	2008-04-08	11
-126	val_126	2008-04-08	11
-90	val_90	2008-04-08	11
-169	val_169	2008-04-08	11
-403	val_403	2008-04-08	11
-400	val_400	2008-04-08	11
-200	val_200	2008-04-08	11
-97	val_97	2008-04-08	11
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 b/sql/hive/src/test/resources/golden/mi-8-e946bdb81b0a831908c1c8196fdff215
deleted file mode 100644
index f132aba9f3ee13e5110e04b4d1a2f01edb212e3b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/mi-8-e946bdb81b0a831908c1c8196fdff215
+++ /dev/null
@@ -1,309 +0,0 @@
-0	val_0	2008-04-08	12
-2	val_2	2008-04-08	12
-4	val_4	2008-04-08	12
-5	val_5	2008-04-08	12
-8	val_8	2008-04-08	12
-9	val_9	2008-04-08	12
-10	val_10	2008-04-08	12
-11	val_11	2008-04-08	12
-12	val_12	2008-04-08	12
-15	val_15	2008-04-08	12
-17	val_17	2008-04-08	12
-18	val_18	2008-04-08	12
-19	val_19	2008-04-08	12
-20	val_20	2008-04-08	12
-24	val_24	2008-04-08	12
-26	val_26	2008-04-08	12
-27	val_27	2008-04-08	12
-28	val_28	2008-04-08	12
-30	val_30	2008-04-08	12
-33	val_33	2008-04-08	12
-34	val_34	2008-04-08	12
-35	val_35	2008-04-08	12
-37	val_37	2008-04-08	12
-41	val_41	2008-04-08	12
-42	val_42	2008-04-08	12
-43	val_43	2008-04-08	12
-44	val_44	2008-04-08	12
-47	val_47	2008-04-08	12
-51	val_51	2008-04-08	12
-53	val_53	2008-04-08	12
-54	val_54	2008-04-08	12
-57	val_57	2008-04-08	12
-58	val_58	2008-04-08	12
-64	val_64	2008-04-08	12
-65	val_65	2008-04-08	12
-66	val_66	2008-04-08	12
-67	val_67	2008-04-08	12
-69	val_69	2008-04-08	12
-70	val_70	2008-04-08	12
-72	val_72	2008-04-08	12
-74	val_74	2008-04-08	12
-76	val_76	2008-04-08	12
-77	val_77	2008-04-08	12
-78	val_78	2008-04-08	12
-80	val_80	2008-04-08	12
-82	val_82	2008-04-08	12
-83	val_83	2008-04-08	12
-84	val_84	2008-04-08	12
-85	val_85	2008-04-08	12
-86	val_86	2008-04-08	12
-87	val_87	2008-04-08	12
-90	val_90	2008-04-08	12
-92	val_92	2008-04-08	12
-95	val_95	2008-04-08	12
-96	val_96	2008-04-08	12
-97	val_97	2008-04-08	12
-98	val_98	2008-04-08	12
-100	val_100	2008-04-08	12
-103	val_103	2008-04-08	12
-104	val_104	2008-04-08	12
-105	val_105	2008-04-08	12
-111	val_111	2008-04-08	12
-113	val_113	2008-04-08	12
-114	val_114	2008-04-08	12
-116	val_116	2008-04-08	12
-118	val_118	2008-04-08	12
-119	val_119	2008-04-08	12
-120	val_120	2008-04-08	12
-125	val_125	2008-04-08	12
-126	val_126	2008-04-08	12
-128	val_128	2008-04-08	12
-129	val_129	2008-04-08	12
-131	val_131	2008-04-08	12
-133	val_133	2008-04-08	12
-134	val_134	2008-04-08	12
-136	val_136	2008-04-08	12
-137	val_137	2008-04-08	12
-138	val_138	2008-04-08	12
-143	val_143	2008-04-08	12
-145	val_145	2008-04-08	12
-146	val_146	2008-04-08	12
-149	val_149	2008-04-08	12
-150	val_150	2008-04-08	12
-152	val_152	2008-04-08	12
-153	val_153	2008-04-08	12
-155	val_155	2008-04-08	12
-156	val_156	2008-04-08	12
-157	val_157	2008-04-08	12
-158	val_158	2008-04-08	12
-160	val_160	2008-04-08	12
-162	val_162	2008-04-08	12
-163	val_163	2008-04-08	12
-164	val_164	2008-04-08	12
-165	val_165	2008-04-08	12
-166	val_166	2008-04-08	12
-167	val_167	2008-04-08	12
-168	val_168	2008-04-08	12
-169	val_169	2008-04-08	12
-170	val_170	2008-04-08	12
-172	val_172	2008-04-08	12
-174	val_174	2008-04-08	12
-175	val_175	2008-04-08	12
-176	val_176	2008-04-08	12
-177	val_177	2008-04-08	12
-178	val_178	2008-04-08	12
-179	val_179	2008-04-08	12
-180	val_180	2008-04-08	12
-181	val_181	2008-04-08	12
-183	val_183	2008-04-08	12
-186	val_186	2008-04-08	12
-187	val_187	2008-04-08	12
-189	val_189	2008-04-08	12
-190	val_190	2008-04-08	12
-191	val_191	2008-04-08	12
-192	val_192	2008-04-08	12
-193	val_193	2008-04-08	12
-194	val_194	2008-04-08	12
-195	val_195	2008-04-08	12
-196	val_196	2008-04-08	12
-197	val_197	2008-04-08	12
-199	val_199	2008-04-08	12
-200	val_200	2008-04-08	12
-201	val_201	2008-04-08	12
-202	val_202	2008-04-08	12
-203	val_203	2008-04-08	12
-205	val_205	2008-04-08	12
-207	val_207	2008-04-08	12
-208	val_208	2008-04-08	12
-209	val_209	2008-04-08	12
-213	val_213	2008-04-08	12
-214	val_214	2008-04-08	12
-216	val_216	2008-04-08	12
-217	val_217	2008-04-08	12
-218	val_218	2008-04-08	12
-219	val_219	2008-04-08	12
-221	val_221	2008-04-08	12
-222	val_222	2008-04-08	12
-223	val_223	2008-04-08	12
-224	val_224	2008-04-08	12
-226	val_226	2008-04-08	12
-228	val_228	2008-04-08	12
-229	val_229	2008-04-08	12
-230	val_230	2008-04-08	12
-233	val_233	2008-04-08	12
-235	val_235	2008-04-08	12
-237	val_237	2008-04-08	12
-238	val_238	2008-04-08	12
-239	val_239	2008-04-08	12
-241	val_241	2008-04-08	12
-242	val_242	2008-04-08	12
-244	val_244	2008-04-08	12
-247	val_247	2008-04-08	12
-248	val_248	2008-04-08	12
-249	val_249	2008-04-08	12
-252	val_252	2008-04-08	12
-255	val_255	2008-04-08	12
-256	val_256	2008-04-08	12
-257	val_257	2008-04-08	12
-258	val_258	2008-04-08	12
-260	val_260	2008-04-08	12
-262	val_262	2008-04-08	12
-263	val_263	2008-04-08	12
-265	val_265	2008-04-08	12
-266	val_266	2008-04-08	12
-272	val_272	2008-04-08	12
-273	val_273	2008-04-08	12
-274	val_274	2008-04-08	12
-275	val_275	2008-04-08	12
-277	val_277	2008-04-08	12
-278	val_278	2008-04-08	12
-280	val_280	2008-04-08	12
-281	val_281	2008-04-08	12
-282	val_282	2008-04-08	12
-283	val_283	2008-04-08	12
-284	val_284	2008-04-08	12
-285	val_285	2008-04-08	12
-286	val_286	2008-04-08	12
-287	val_287	2008-04-08	12
-288	val_288	2008-04-08	12
-289	val_289	2008-04-08	12
-291	val_291	2008-04-08	12
-292	val_292	2008-04-08	12
-296	val_296	2008-04-08	12
-298	val_298	2008-04-08	12
-302	val_302	2008-04-08	12
-305	val_305	2008-04-08	12
-306	val_306	2008-04-08	12
-307	val_307	2008-04-08	12
-308	val_308	2008-04-08	12
-309	val_309	2008-04-08	12
-310	val_310	2008-04-08	12
-311	val_311	2008-04-08	12
-315	val_315	2008-04-08	12
-316	val_316	2008-04-08	12
-317	val_317	2008-04-08	12
-318	val_318	2008-04-08	12
-321	val_321	2008-04-08	12
-322	val_322	2008-04-08	12
-323	val_323	2008-04-08	12
-325	val_325	2008-04-08	12
-327	val_327	2008-04-08	12
-331	val_331	2008-04-08	12
-332	val_332	2008-04-08	12
-333	val_333	2008-04-08	12
-335	val_335	2008-04-08	12
-336	val_336	2008-04-08	12
-338	val_338	2008-04-08	12
-339	val_339	2008-04-08	12
-341	val_341	2008-04-08	12
-342	val_342	2008-04-08	12
-344	val_344	2008-04-08	12
-345	val_345	2008-04-08	12
-348	val_348	2008-04-08	12
-351	val_351	2008-04-08	12
-353	val_353	2008-04-08	12
-356	val_356	2008-04-08	12
-360	val_360	2008-04-08	12
-362	val_362	2008-04-08	12
-364	val_364	2008-04-08	12
-365	val_365	2008-04-08	12
-366	val_366	2008-04-08	12
-367	val_367	2008-04-08	12
-368	val_368	2008-04-08	12
-369	val_369	2008-04-08	12
-373	val_373	2008-04-08	12
-374	val_374	2008-04-08	12
-375	val_375	2008-04-08	12
-377	val_377	2008-04-08	12
-378	val_378	2008-04-08	12
-379	val_379	2008-04-08	12
-382	val_382	2008-04-08	12
-384	val_384	2008-04-08	12
-386	val_386	2008-04-08	12
-389	val_389	2008-04-08	12
-392	val_392	2008-04-08	12
-393	val_393	2008-04-08	12
-394	val_394	2008-04-08	12
-395	val_395	2008-04-08	12
-396	val_396	2008-04-08	12
-397	val_397	2008-04-08	12
-399	val_399	2008-04-08	12
-400	val_400	2008-04-08	12
-401	val_401	2008-04-08	12
-402	val_402	2008-04-08	12
-403	val_403	2008-04-08	12
-404	val_404	2008-04-08	12
-406	val_406	2008-04-08	12
-407	val_407	2008-04-08	12
-409	val_409	2008-04-08	12
-411	val_411	2008-04-08	12
-413	val_413	2008-04-08	12
-414	val_414	2008-04-08	12
-417	val_417	2008-04-08	12
-418	val_418	2008-04-08	12
-419	val_419	2008-04-08	12
-421	val_421	2008-04-08	12
-424	val_424	2008-04-08	12
-427	val_427	2008-04-08	12
-429	val_429	2008-04-08	12
-430	val_430	2008-04-08	12
-431	val_431	2008-04-08	12
-432	val_432	2008-04-08	12
-435	val_435	2008-04-08	12
-436	val_436	2008-04-08	12
-437	val_437	2008-04-08	12
-438	val_438	2008-04-08	12
-439	val_439	2008-04-08	12
-443	val_443	2008-04-08	12
-444	val_444	2008-04-08	12
-446	val_446	2008-04-08	12
-448	val_448	2008-04-08	12
-449	val_449	2008-04-08	12
-452	val_452	2008-04-08	12
-453	val_453	2008-04-08	12
-454	val_454	2008-04-08	12
-455	val_455	2008-04-08	12
-457	val_457	2008-04-08	12
-458	val_458	2008-04-08	12
-459	val_459	2008-04-08	12
-460	val_460	2008-04-08	12
-462	val_462	2008-04-08	12
-463	val_463	2008-04-08	12
-466	val_466	2008-04-08	12
-467	val_467	2008-04-08	12
-468	val_468	2008-04-08	12
-469	val_469	2008-04-08	12
-470	val_470	2008-04-08	12
-472	val_472	2008-04-08	12
-475	val_475	2008-04-08	12
-477	val_477	2008-04-08	12
-478	val_478	2008-04-08	12
-479	val_479	2008-04-08	12
-480	val_480	2008-04-08	12
-481	val_481	2008-04-08	12
-482	val_482	2008-04-08	12
-483	val_483	2008-04-08	12
-484	val_484	2008-04-08	12
-485	val_485	2008-04-08	12
-487	val_487	2008-04-08	12
-489	val_489	2008-04-08	12
-490	val_490	2008-04-08	12
-491	val_491	2008-04-08	12
-492	val_492	2008-04-08	12
-493	val_493	2008-04-08	12
-494	val_494	2008-04-08	12
-495	val_495	2008-04-08	12
-496	val_496	2008-04-08	12
-497	val_497	2008-04-08	12
-498	val_498	2008-04-08	12
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66 b/sql/hive/src/test/resources/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66
index 55b22b1aa9f24be59e8f86153e626c19e03f575e..4eb9d0d9dbd22693f9e87417c047bc97df268ebf 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66
@@ -1 +1 @@
-580
\ No newline at end of file
+580
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 b/sql/hive/src/test/resources/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3
index 55b22b1aa9f24be59e8f86153e626c19e03f575e..4eb9d0d9dbd22693f9e87417c047bc97df268ebf 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3
@@ -1 +1 @@
-580
\ No newline at end of file
+580
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee b/sql/hive/src/test/resources/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee
index fa8ef14301a13098160aac51f2be68a1273ac9eb..7eb898446847ae19fc4a634d6ea9e8281d1ade0b 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee
@@ -3,4 +3,4 @@
 10
 270
 10
-10
\ No newline at end of file
+10
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 b/sql/hive/src/test/resources/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 b/sql/hive/src/test/resources/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1
index a3a38a80b791062ee6292ef9902c87e44938d63e..654cce85277c0878782baaad56fe7aab95de3f74 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1
@@ -1 +1 @@
-247580	247580	247580	247580	247580	247580	548662743780	548662743780
\ No newline at end of file
+247580	247580	247580	247580	247580	247580	548662743780	548662743780
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 b/sql/hive/src/test/resources/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1
index a3a38a80b791062ee6292ef9902c87e44938d63e..654cce85277c0878782baaad56fe7aab95de3f74 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1
@@ -1 +1 @@
-247580	247580	247580	247580	247580	247580	548662743780	548662743780
\ No newline at end of file
+247580	247580	247580	247580	247580	247580	548662743780	548662743780
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f b/sql/hive/src/test/resources/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 b/sql/hive/src/test/resources/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1
index a3a38a80b791062ee6292ef9902c87e44938d63e..654cce85277c0878782baaad56fe7aab95de3f74 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1
@@ -1 +1 @@
-247580	247580	247580	247580	247580	247580	548662743780	548662743780
\ No newline at end of file
+247580	247580	247580	247580	247580	247580	548662743780	548662743780
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 b/sql/hive/src/test/resources/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 b/sql/hive/src/test/resources/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1
index a3a38a80b791062ee6292ef9902c87e44938d63e..654cce85277c0878782baaad56fe7aab95de3f74 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1
+++ b/sql/hive/src/test/resources/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1
@@ -1 +1 @@
-247580	247580	247580	247580	247580	247580	548662743780	548662743780
\ No newline at end of file
+247580	247580	247580	247580	247580	247580	548662743780	548662743780
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8 b/sql/hive/src/test/resources/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8
index 3c505d9663010fdbd1431bda325f15d9913fbe0c..d5bad34b0246ef72902d97b32dc811a57119fcc3 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8
@@ -50,4 +50,4 @@ NULL
 406
 406
 406
-406
\ No newline at end of file
+406
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe b/sql/hive/src/test/resources/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe
index a877195ef3ab17b2af4e426dab223406f331ba66..d00aeb4be034016010b35d41c178a4fc68375c5e 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe
@@ -12,4 +12,4 @@
 311	1
 369	1
 401	1
-406	1
\ No newline at end of file
+406	1
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe b/sql/hive/src/test/resources/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe
index a877195ef3ab17b2af4e426dab223406f331ba66..d00aeb4be034016010b35d41c178a4fc68375c5e 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe
@@ -12,4 +12,4 @@
 311	1
 369	1
 401	1
-406	1
\ No newline at end of file
+406	1
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56 b/sql/hive/src/test/resources/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56
index a877195ef3ab17b2af4e426dab223406f331ba66..d00aeb4be034016010b35d41c178a4fc68375c5e 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56
@@ -12,4 +12,4 @@
 311	1
 369	1
 401	1
-406	1
\ No newline at end of file
+406	1
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 b/sql/hive/src/test/resources/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56 b/sql/hive/src/test/resources/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56
index a877195ef3ab17b2af4e426dab223406f331ba66..d00aeb4be034016010b35d41c178a4fc68375c5e 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56
@@ -12,4 +12,4 @@
 311	1
 369	1
 401	1
-406	1
\ No newline at end of file
+406	1
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e b/sql/hive/src/test/resources/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e
index 25bf17fc5aaabd17402e77a2b16f95fbea7310d2..3c032078a4a21c5c51d3c93d91717c1dabbb8cd0 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e
@@ -1 +1 @@
-18
\ No newline at end of file
+18
diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/multiMapJoin2-28-c14792ccac2ca64e3e9e21af4fd12d2c
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/multiMapJoin2-28-c14792ccac2ca64e3e9e21af4fd12d2c
diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/multiMapJoin2-29-b9d963d24994c47c3776dda6f7d3881f
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/multiMapJoin2-29-b9d963d24994c47c3776dda6f7d3881f
diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/multiMapJoin2-30-6d1c7f7014fc709148b0f401c5f23cb3
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a
rename to sql/hive/src/test/resources/golden/multiMapJoin2-30-6d1c7f7014fc709148b0f401c5f23cb3
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-31-2e9c4d5e2bd709e96f311488ada116b0 b/sql/hive/src/test/resources/golden/multiMapJoin2-31-2e9c4d5e2bd709e96f311488ada116b0
new file mode 100644
index 0000000000000000000000000000000000000000..d07c26f55176d30340cd7f4eaf8e84f221072b54
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-31-2e9c4d5e2bd709e96f311488ada116b0
@@ -0,0 +1,1000 @@
+0
+0
+0
+2
+4
+5
+5
+5
+8
+9
+10
+11
+12
+12
+15
+15
+17
+18
+18
+19
+20
+24
+24
+26
+26
+27
+28
+30
+33
+34
+35
+35
+35
+37
+37
+41
+42
+42
+43
+44
+47
+51
+51
+53
+54
+57
+58
+58
+64
+65
+66
+67
+67
+69
+70
+70
+70
+72
+72
+74
+76
+76
+77
+78
+80
+82
+83
+83
+84
+84
+85
+86
+87
+90
+90
+90
+92
+95
+95
+96
+97
+97
+98
+98
+100
+100
+103
+103
+104
+104
+105
+111
+113
+113
+114
+116
+118
+118
+119
+119
+119
+120
+120
+125
+125
+126
+128
+128
+128
+129
+129
+131
+133
+134
+134
+136
+137
+137
+138
+138
+138
+138
+143
+145
+146
+146
+149
+149
+150
+152
+152
+153
+155
+156
+157
+158
+160
+162
+163
+164
+164
+165
+165
+166
+167
+167
+167
+168
+169
+169
+169
+169
+170
+172
+172
+174
+174
+175
+175
+176
+176
+177
+178
+179
+179
+180
+181
+183
+186
+187
+187
+187
+189
+190
+191
+191
+192
+193
+193
+193
+194
+195
+195
+196
+197
+197
+199
+199
+199
+200
+200
+201
+202
+203
+203
+205
+205
+207
+207
+208
+208
+208
+209
+209
+213
+213
+214
+216
+216
+217
+217
+218
+219
+219
+221
+221
+222
+223
+223
+224
+224
+226
+228
+229
+229
+230
+230
+230
+230
+230
+233
+233
+235
+237
+237
+238
+238
+239
+239
+241
+242
+242
+244
+247
+248
+249
+252
+255
+255
+256
+256
+257
+258
+260
+262
+263
+265
+265
+266
+272
+272
+273
+273
+273
+274
+275
+277
+277
+277
+277
+278
+278
+280
+280
+281
+281
+282
+282
+283
+284
+285
+286
+287
+288
+288
+289
+291
+292
+296
+298
+298
+298
+302
+305
+306
+307
+307
+308
+309
+309
+310
+311
+311
+311
+315
+316
+316
+316
+317
+317
+318
+318
+318
+321
+321
+322
+322
+323
+325
+325
+327
+327
+327
+331
+331
+332
+333
+333
+335
+336
+338
+339
+341
+342
+342
+344
+344
+345
+348
+348
+348
+348
+348
+351
+353
+353
+356
+360
+362
+364
+365
+366
+367
+367
+368
+369
+369
+369
+373
+374
+375
+377
+378
+379
+382
+382
+384
+384
+384
+386
+389
+392
+393
+394
+395
+395
+396
+396
+396
+397
+397
+399
+399
+400
+401
+401
+401
+401
+401
+402
+403
+403
+403
+404
+404
+406
+406
+406
+406
+407
+409
+409
+409
+411
+413
+413
+414
+414
+417
+417
+417
+418
+419
+421
+424
+424
+427
+429
+429
+430
+430
+430
+431
+431
+431
+432
+435
+436
+437
+438
+438
+438
+439
+439
+443
+444
+446
+448
+449
+452
+453
+454
+454
+454
+455
+457
+458
+458
+459
+459
+460
+462
+462
+463
+463
+466
+466
+466
+467
+468
+468
+468
+468
+469
+469
+469
+469
+469
+470
+472
+475
+477
+478
+478
+479
+480
+480
+480
+481
+482
+483
+484
+485
+487
+489
+489
+489
+489
+490
+491
+492
+492
+493
+494
+495
+496
+497
+498
+498
+498
+0
+0
+0
+2
+4
+5
+5
+5
+8
+9
+10
+11
+12
+12
+15
+15
+17
+18
+18
+19
+20
+24
+24
+26
+26
+27
+28
+30
+33
+34
+35
+35
+35
+37
+37
+41
+42
+42
+43
+44
+47
+51
+51
+53
+54
+57
+58
+58
+64
+65
+66
+67
+67
+69
+70
+70
+70
+72
+72
+74
+76
+76
+77
+78
+80
+82
+83
+83
+84
+84
+85
+86
+87
+90
+90
+90
+92
+95
+95
+96
+97
+97
+98
+98
+100
+100
+103
+103
+104
+104
+105
+111
+113
+113
+114
+116
+118
+118
+119
+119
+119
+120
+120
+125
+125
+126
+128
+128
+128
+129
+129
+131
+133
+134
+134
+136
+137
+137
+138
+138
+138
+138
+143
+145
+146
+146
+149
+149
+150
+152
+152
+153
+155
+156
+157
+158
+160
+162
+163
+164
+164
+165
+165
+166
+167
+167
+167
+168
+169
+169
+169
+169
+170
+172
+172
+174
+174
+175
+175
+176
+176
+177
+178
+179
+179
+180
+181
+183
+186
+187
+187
+187
+189
+190
+191
+191
+192
+193
+193
+193
+194
+195
+195
+196
+197
+197
+199
+199
+199
+200
+200
+201
+202
+203
+203
+205
+205
+207
+207
+208
+208
+208
+209
+209
+213
+213
+214
+216
+216
+217
+217
+218
+219
+219
+221
+221
+222
+223
+223
+224
+224
+226
+228
+229
+229
+230
+230
+230
+230
+230
+233
+233
+235
+237
+237
+238
+238
+239
+239
+241
+242
+242
+244
+247
+248
+249
+252
+255
+255
+256
+256
+257
+258
+260
+262
+263
+265
+265
+266
+272
+272
+273
+273
+273
+274
+275
+277
+277
+277
+277
+278
+278
+280
+280
+281
+281
+282
+282
+283
+284
+285
+286
+287
+288
+288
+289
+291
+292
+296
+298
+298
+298
+302
+305
+306
+307
+307
+308
+309
+309
+310
+311
+311
+311
+315
+316
+316
+316
+317
+317
+318
+318
+318
+321
+321
+322
+322
+323
+325
+325
+327
+327
+327
+331
+331
+332
+333
+333
+335
+336
+338
+339
+341
+342
+342
+344
+344
+345
+348
+348
+348
+348
+348
+351
+353
+353
+356
+360
+362
+364
+365
+366
+367
+367
+368
+369
+369
+369
+373
+374
+375
+377
+378
+379
+382
+382
+384
+384
+384
+386
+389
+392
+393
+394
+395
+395
+396
+396
+396
+397
+397
+399
+399
+400
+401
+401
+401
+401
+401
+402
+403
+403
+403
+404
+404
+406
+406
+406
+406
+407
+409
+409
+409
+411
+413
+413
+414
+414
+417
+417
+417
+418
+419
+421
+424
+424
+427
+429
+429
+430
+430
+430
+431
+431
+431
+432
+435
+436
+437
+438
+438
+438
+439
+439
+443
+444
+446
+448
+449
+452
+453
+454
+454
+454
+455
+457
+458
+458
+459
+459
+460
+462
+462
+463
+463
+466
+466
+466
+467
+468
+468
+468
+468
+469
+469
+469
+469
+469
+470
+472
+475
+477
+478
+478
+479
+480
+480
+480
+481
+482
+483
+484
+485
+487
+489
+489
+489
+489
+490
+491
+492
+492
+493
+494
+495
+496
+497
+498
+498
+498
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656 b/sql/hive/src/test/resources/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656
index 8984af9eb83ddf68b0303fb999e187eadbfff4ec..be996088edb51da0eebc3168890e23f86dd4f8c2 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656
@@ -71,4 +71,4 @@
 406
 406
 406
-406
\ No newline at end of file
+406
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 b/sql/hive/src/test/resources/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656 b/sql/hive/src/test/resources/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656
index 8984af9eb83ddf68b0303fb999e187eadbfff4ec..be996088edb51da0eebc3168890e23f86dd4f8c2 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656
@@ -71,4 +71,4 @@
 406
 406
 406
-406
\ No newline at end of file
+406
diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 b/sql/hive/src/test/resources/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6
+++ b/sql/hive/src/test/resources/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69
index 0e3cc2ad6a4e69ac7a4f2af5ffe2780aebaf6c77..891c047cfaa8a792440bcb50d394ee7b46b223b9 100644
--- a/sql/hive/src/test/resources/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69
+++ b/sql/hive/src/test/resources/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69
@@ -34,4 +34,4 @@
 495	1
 496	1
 497	1
-498	3
\ No newline at end of file
+498	3
diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69
index 0e3cc2ad6a4e69ac7a4f2af5ffe2780aebaf6c77..891c047cfaa8a792440bcb50d394ee7b46b223b9 100644
--- a/sql/hive/src/test/resources/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69
+++ b/sql/hive/src/test/resources/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69
@@ -34,4 +34,4 @@
 495	1
 496	1
 497	1
-498	3
\ No newline at end of file
+498	3
diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63
index df07a9da29f01512947648bed0e2a4f71e8798dc..93e965c7714038926c1e1173addaeed83556567f 100644
--- a/sql/hive/src/test/resources/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63
+++ b/sql/hive/src/test/resources/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63
@@ -306,4 +306,4 @@
 495	1
 496	1
 497	1
-498	3
\ No newline at end of file
+498	3
diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69 b/sql/hive/src/test/resources/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 b/sql/hive/src/test/resources/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 b/sql/hive/src/test/resources/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e b/sql/hive/src/test/resources/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69
deleted file mode 100644
index eb1f49486af7c892e115f610fa32b505125766fc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69
+++ /dev/null
@@ -1 +0,0 @@
-500
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63
index e8804e62d0aaffd284bff32530c0e7f1b4d8282a..3c5c916a9fbccca538e1bcc81b927262d3520439 100644
--- a/sql/hive/src/test/resources/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63
+++ b/sql/hive/src/test/resources/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63
@@ -306,4 +306,4 @@
 495	495.0	val_495
 496	496.0	val_496
 497	497.0	val_497
-498	1494.0	val_498
\ No newline at end of file
+498	1494.0	val_498
diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69
index fc0b77c7a362808dd09a0213e82c84fc4477a956..4494c27e47f59ad2bf56dc2390b4d4a1644d3ad6 100644
--- a/sql/hive/src/test/resources/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69
+++ b/sql/hive/src/test/resources/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69
@@ -306,4 +306,4 @@
 495	1.0
 496	1.0
 497	1.0
-498	1.0
\ No newline at end of file
+498	1.0
diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63
index e8804e62d0aaffd284bff32530c0e7f1b4d8282a..3c5c916a9fbccca538e1bcc81b927262d3520439 100644
--- a/sql/hive/src/test/resources/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63
+++ b/sql/hive/src/test/resources/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63
@@ -306,4 +306,4 @@
 495	495.0	val_495
 496	496.0	val_496
 497	497.0	val_497
-498	1494.0	val_498
\ No newline at end of file
+498	1494.0	val_498
diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69
index fc0b77c7a362808dd09a0213e82c84fc4477a956..4494c27e47f59ad2bf56dc2390b4d4a1644d3ad6 100644
--- a/sql/hive/src/test/resources/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69
+++ b/sql/hive/src/test/resources/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69
@@ -306,4 +306,4 @@
 495	1.0
 496	1.0
 497	1.0
-498	1.0
\ No newline at end of file
+498	1.0
diff --git a/sql/hive/src/test/resources/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6 b/sql/hive/src/test/resources/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 b/sql/hive/src/test/resources/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364
deleted file mode 100644
index c10c17978a0d81aa1e1b9a1260f3342cc927f1d0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364
+++ /dev/null
@@ -1,5 +0,0 @@
-simple_int          	int                 	None                
-max_nested_array    	array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<int>>>>>>>>>>>>>>>>>>>>>>>	None                
-max_nested_map      	array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<map<string,string>>>>>>>>>>>>>>>>>>>>>>	None                
-max_nested_struct   	array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<struct<s:string,i:bigint>>>>>>>>>>>>>>>>>>>>>>>	None                
-simple_string       	string              	None                
diff --git a/sql/hive/src/test/resources/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 b/sql/hive/src/test/resources/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9
deleted file mode 100644
index 8ce70d8c71782c813931e8609f91543e1af449cf..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9
+++ /dev/null
@@ -1,8 +0,0 @@
-simple_int          	int                 	None                
-max_nested_array    	array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<int>>>>>>>>>>>>>>>>>>>>>>>	None                
-max_nested_map      	array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<map<string,string>>>>>>>>>>>>>>>>>>>>>>	None                
-max_nested_struct   	array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<struct<s:string,i:bigint>>>>>>>>>>>>>>>>>>>>>>>	None                
-simple_string       	string              	None                
-	 	 
-Detailed Table Information	Table(tableName:nestedcomplex, dbName:default, owner:marmbrus, createTime:1391226936, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:simple_int, type:int, comment:null), FieldSchema(name:max_nested_array, type:array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<int>>>>>>>>>>>>>>>>>>>>>>>, comment:null), FieldSchema(name:max_nested_map, type:array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<map<string,string>>>>>>>>>>>>>>>>>>>>>>, comment:null), FieldSchema(name:max_nested_struct, type:array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<array<struct<s:string,i:bigint>>>>>>>>>>>>>>>>>>>>>>>, comment:null), FieldSchema(name:simple_string, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/nestedcomplex, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1, line.delim=	 
-, hive.serialization.extend.nesting.levels=true}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1391226936}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)		 
diff --git a/sql/hive/src/test/resources/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 b/sql/hive/src/test/resources/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d b/sql/hive/src/test/resources/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d
deleted file mode 100644
index 228853cffe52707f9edaba68a5f0f049daccfba2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d
+++ /dev/null
@@ -1,2 +0,0 @@
-2	[[[[[[[[[[[[[[[[[[[[[[[0,3,2]]]]]]]]]]]]]]]]]]]]]]]	[[[[[[[[[[[[[[[[[[[[[{"k1":"v1","k3":"v3"}]]]]]]]]]]]]]]]]]]]]]	[[[[[[[[[[[[[[[[[[[[[[{"s":"b","i":10}]]]]]]]]]]]]]]]]]]]]]]	2
-3	[[[[[[[[[[[[[[[[[[[[[[[0,1,2]]]]]]]]]]]]]]]]]]]]]]]	[[[[[[[[[[[[[[[[[[[[[{"k1":"v1","k2":"v2"}]]]]]]]]]]]]]]]]]]]]]	[[[[[[[[[[[[[[[[[[[[[[{"s":"a","i":10}]]]]]]]]]]]]]]]]]]]]]]	2
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 b/sql/hive/src/test/resources/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f b/sql/hive/src/test/resources/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 b/sql/hive/src/test/resources/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 b/sql/hive/src/test/resources/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 b/sql/hive/src/test/resources/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f b/sql/hive/src/test/resources/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 b/sql/hive/src/test/resources/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f b/sql/hive/src/test/resources/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 b/sql/hive/src/test/resources/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 b/sql/hive/src/test/resources/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 b/sql/hive/src/test/resources/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f b/sql/hive/src/test/resources/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 b/sql/hive/src/test/resources/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 b/sql/hive/src/test/resources/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 b/sql/hive/src/test/resources/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f b/sql/hive/src/test/resources/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 b/sql/hive/src/test/resources/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f b/sql/hive/src/test/resources/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/newline-2-4eb54a664e549614d56ca088c8867d b/sql/hive/src/test/resources/golden/newline-2-4eb54a664e549614d56ca088c8867d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 b/sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6
index cdcd12dca3ca27c40f835d93786636370b5a887c..9c4a8fd1314074dd3fe9edbce0930ac4eb2042fb 100644
--- a/sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6
+++ b/sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6
@@ -81,4 +81,4 @@ val_84
 val_28
 val_37
 val_90
-val_97
\ No newline at end of file
+val_97
diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b
+++ b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716
index a0cfcf2621f2e747b420b574443fc3c63733badc..34e8c1127f0162d67222c39d7c3073a4f8177cc6 100644
--- a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716
+++ b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716
@@ -22,4 +22,4 @@ NULL	val_484	25
 311	val_311	25
 369		25
 401	val_401	25
-406	val_406	25
\ No newline at end of file
+406	val_406	25
diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716
index a0cfcf2621f2e747b420b574443fc3c63733badc..34e8c1127f0162d67222c39d7c3073a4f8177cc6 100644
--- a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716
+++ b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716
@@ -22,4 +22,4 @@ NULL	val_484	25
 311	val_311	25
 369		25
 401	val_401	25
-406	val_406	25
\ No newline at end of file
+406	val_406	25
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 b/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 b/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092
deleted file mode 100644
index 24e19ec6caa8f211c419b5a3367fe9ec7e8aa4f2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092
+++ /dev/null
@@ -1 +0,0 @@
--826625916
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9
deleted file mode 100644
index 05b0c43f926bd30d9e1464b36e243dde394f0452..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9
+++ /dev/null
@@ -1 +0,0 @@
-10226524244
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296
deleted file mode 100644
index bc15b9ca549855b31ac7cc9d530952522dc28070..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296
+++ /dev/null
@@ -1 +0,0 @@
-20453048488
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad
deleted file mode 100644
index 08839f6bb296e888d311d8ea2f35f7ff82dd3f2d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad
+++ /dev/null
@@ -1 +0,0 @@
-200
diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f
index fce122031e88f6f605faedaa054d99cbe1393287..138522d542895bcd66a3ec13eb24806899b8760c 100644
--- a/sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -54,4 +54,4 @@
 1234	95	2.0
 1234	96	1.0
 1234	97	2.0
-1234	98	2.0
\ No newline at end of file
+1234	98	2.0
diff --git a/sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f
index fce122031e88f6f605faedaa054d99cbe1393287..138522d542895bcd66a3ec13eb24806899b8760c 100644
--- a/sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f
@@ -54,4 +54,4 @@
 1234	95	2.0
 1234	96	1.0
 1234	97	2.0
-1234	98	2.0
\ No newline at end of file
+1234	98	2.0
diff --git a/sql/hive/src/test/resources/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 b/sql/hive/src/test/resources/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c b/sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c
deleted file mode 100644
index d111428eaabb05caf7a5f7f7f31ee2322c968f3e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c
+++ /dev/null
@@ -1 +0,0 @@
-[null,0]	[null,[]]	[null,{}]	[null,{"col1":0}]
diff --git a/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302
+++ b/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302
+++ b/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302
+++ b/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302
+++ b/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063
+++ b/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66
+++ b/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f
+++ b/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/nullgroup3-1-61ead7f73d525e0d9e21beba91a3d39e
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394
rename to sql/hive/src/test/resources/golden/nullgroup3-1-61ead7f73d525e0d9e21beba91a3d39e
diff --git a/sql/hive/src/test/resources/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 b/sql/hive/src/test/resources/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/nullgroup3-12-61ead7f73d525e0d9e21beba91a3d39e
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1
rename to sql/hive/src/test/resources/golden/nullgroup3-12-61ead7f73d525e0d9e21beba91a3d39e
diff --git a/sql/hive/src/test/resources/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 b/sql/hive/src/test/resources/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/nullgroup3-13-a5e12cfbc1799ce9fa9628d81b8c0b06
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77
rename to sql/hive/src/test/resources/golden/nullgroup3-13-a5e12cfbc1799ce9fa9628d81b8c0b06
diff --git a/sql/hive/src/test/resources/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb b/sql/hive/src/test/resources/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 b/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280
+++ b/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/nullgroup3-18-113e2b587784d54d2a5b5f091389397e
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd
rename to sql/hive/src/test/resources/golden/nullgroup3-18-113e2b587784d54d2a5b5f091389397e
diff --git a/sql/hive/src/test/resources/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 b/sql/hive/src/test/resources/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/nullgroup3-19-77de4b2d65eee228848625cdd422178d
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77
rename to sql/hive/src/test/resources/golden/nullgroup3-19-77de4b2d65eee228848625cdd422178d
diff --git a/sql/hive/src/test/resources/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 b/sql/hive/src/test/resources/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/nullgroup3-2-a5e12cfbc1799ce9fa9628d81b8c0b06
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd
rename to sql/hive/src/test/resources/golden/nullgroup3-2-a5e12cfbc1799ce9fa9628d81b8c0b06
diff --git a/sql/hive/src/test/resources/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb b/sql/hive/src/test/resources/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 b/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6
+++ b/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 b/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280
+++ b/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/nullgroup3-6-113e2b587784d54d2a5b5f091389397e
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77
rename to sql/hive/src/test/resources/golden/nullgroup3-6-113e2b587784d54d2a5b5f091389397e
diff --git a/sql/hive/src/test/resources/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 b/sql/hive/src/test/resources/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/nullgroup3-7-77de4b2d65eee228848625cdd422178d
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd
rename to sql/hive/src/test/resources/golden/nullgroup3-7-77de4b2d65eee228848625cdd422178d
diff --git a/sql/hive/src/test/resources/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 b/sql/hive/src/test/resources/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 b/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6
+++ b/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/nullgroup5-1-642e12a05bf01a6716bfa15ed0012629
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77
rename to sql/hive/src/test/resources/golden/nullgroup5-1-642e12a05bf01a6716bfa15ed0012629
diff --git a/sql/hive/src/test/resources/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b b/sql/hive/src/test/resources/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/nullgroup5-3-35517978e09aa1bd1d15f51d11e08fd5
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd
rename to sql/hive/src/test/resources/golden/nullgroup5-3-35517978e09aa1bd1d15f51d11e08fd5
diff --git a/sql/hive/src/test/resources/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c b/sql/hive/src/test/resources/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c b/sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c
index c5c8d29fdd13e6dee2330bc99daedf917717dbbb..7aae61e5eb82f6ed0a954ef086bae93dca6546e8 100644
--- a/sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c
+++ b/sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c
@@ -497,4 +497,4 @@
 403	val_403
 400	val_400
 200	val_200
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 b/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2
+++ b/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 b/sql/hive/src/test/resources/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/nullscript-1-f07dfd6ef687e038083deca5941d8174
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/nullscript-1-f07dfd6ef687e038083deca5941d8174
diff --git a/sql/hive/src/test/resources/golden/nullscript-2-17238164053203d56d30704e2c098e80 b/sql/hive/src/test/resources/golden/nullscript-2-17238164053203d56d30704e2c098e80
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/nullscript-2-bb1abcf2f4a2a5cd5c058104901627bb
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc
rename to sql/hive/src/test/resources/golden/nullscript-2-bb1abcf2f4a2a5cd5c058104901627bb
diff --git a/sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 b/sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710
index d23e05acf7ba58c8471721a7bb93d585e57e75c8..e34118512c1d769075d0b492b217316096597c08 100644
--- a/sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710
+++ b/sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710
@@ -497,4 +497,4 @@
 403
 400
 200
-97
\ No newline at end of file
+97
diff --git a/sql/hive/src/test/resources/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc b/sql/hive/src/test/resources/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 b/sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8
deleted file mode 100644
index da0ddb96957e12b625b8f594c2f7f6220cfe110b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8
+++ /dev/null
@@ -1 +0,0 @@
-NULL	NULL	NULL	1	0	0.0
diff --git a/sql/hive/src/test/resources/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 b/sql/hive/src/test/resources/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d b/sql/hive/src/test/resources/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187
deleted file mode 100644
index 7541739b4860872060e81b9d8489ec08b9764d25..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Fri Jan 31 19:56:27 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1391226987          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 b/sql/hive/src/test/resources/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187
deleted file mode 100644
index 78c0010bebbc6ad3f65da61ba64207dfe764c787..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187
+++ /dev/null
@@ -1,34 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	from deserializer   
-value               	string              	from deserializer   
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Fri Jan 31 19:56:27 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	last_modified_by    	marmbrus            
-	last_modified_time  	1391226987          
-	transient_lastDdlTime	1391226987          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.ql.io.orc.OrcSerde	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.orc.OrcInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d b/sql/hive/src/test/resources/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187
deleted file mode 100644
index 6881640dd3cd44cb92e17a5388305a3b5e46c84b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	from deserializer   
-value               	string              	from deserializer   
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Fri Jan 31 19:56:27 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1391226987          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.ql.io.orc.OrcSerde	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.orc.OrcInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 b/sql/hive/src/test/resources/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 b/sql/hive/src/test/resources/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-20-176d469a0edba57404416535c7d48023 b/sql/hive/src/test/resources/golden/orc_create-20-176d469a0edba57404416535c7d48023
deleted file mode 100644
index dd4fa77f326ad64e671a3034f19d48f269d3fa9f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_create-20-176d469a0edba57404416535c7d48023
+++ /dev/null
@@ -1,29 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-str                 	string              	from deserializer   
-mp                  	map<string,string>  	from deserializer   
-lst                 	array<string>       	from deserializer   
-strct               	struct<A:string,B:string>	from deserializer   
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Fri Jan 31 19:56:27 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create_complex	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1391226987          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.ql.io.orc.OrcSerde	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.orc.OrcInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 b/sql/hive/src/test/resources/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 b/sql/hive/src/test/resources/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6
deleted file mode 100644
index 88c8812029d8287a15e952d38e75d0ba6ccc6274..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6
+++ /dev/null
@@ -1,3 +0,0 @@
-line1	{"key11":"value11","key12":"value12","key13":"value13"}	["a","b","c"]	{"a":"one","b":"two"}
-line2	{"key21":"value21","key22":"value22","key23":"value23"}	["d","e","f"]	{"a":"three","b":"four"}
-line3	{"key31":"value31","key32":"value32","key33":"value33"}	["g","h","i"]	{"a":"five","b":"six"}
diff --git a/sql/hive/src/test/resources/golden/orc_create-23-be779533ea8967231e644209114c8350 b/sql/hive/src/test/resources/golden/orc_create-23-be779533ea8967231e644209114c8350
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 b/sql/hive/src/test/resources/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2
deleted file mode 100644
index 67946888f1baf5469bfa8525f6d4a0a0d42c0c31..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2
+++ /dev/null
@@ -1,3 +0,0 @@
-line1	{"key12":"value12","key11":"value11","key13":"value13"}	["a","b","c"]	{"A":"one","B":"two"}
-line2	{"key21":"value21","key23":"value23","key22":"value22"}	["d","e","f"]	{"A":"three","B":"four"}
-line3	{"key33":"value33","key31":"value31","key32":"value32"}	["g","h","i"]	{"A":"five","B":"six"}
diff --git a/sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead b/sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead
deleted file mode 100644
index 83db48f84ec878fbfb30b46d16630e944e34f205..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead
+++ /dev/null
@@ -1,3 +0,0 @@
-line1
-line2
-line3
diff --git a/sql/hive/src/test/resources/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c b/sql/hive/src/test/resources/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c
deleted file mode 100644
index db4876dd3c80904e42e570bf5e7b195b1880828c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c
+++ /dev/null
@@ -1,3 +0,0 @@
-{"key12":"value12","key11":"value11","key13":"value13"}
-{"key21":"value21","key23":"value23","key22":"value22"}
-{"key33":"value33","key31":"value31","key32":"value32"}
diff --git a/sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 b/sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89
deleted file mode 100644
index 3b2e1cf7e0098f178f47f88cd0de3e71e32b0714..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89
+++ /dev/null
@@ -1,3 +0,0 @@
-["a","b","c"]
-["d","e","f"]
-["g","h","i"]
diff --git a/sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 b/sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2
deleted file mode 100644
index 0f890dfd226d6e83b1e17bdfa1b00e0bd8089dd5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2
+++ /dev/null
@@ -1,3 +0,0 @@
-{"a":"one","b":"two"}
-{"a":"three","b":"four"}
-{"a":"five","b":"six"}
diff --git a/sql/hive/src/test/resources/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 b/sql/hive/src/test/resources/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae b/sql/hive/src/test/resources/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d b/sql/hive/src/test/resources/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 b/sql/hive/src/test/resources/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 b/sql/hive/src/test/resources/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a b/sql/hive/src/test/resources/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a
deleted file mode 100644
index 0c6f5324886071196d79783094d5b0e066e4fbaa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a
+++ /dev/null
@@ -1,10 +0,0 @@
-91	Genevieve	Wilkins	908 Turpis. Street
-93	Mariko	Cline	P.O. Box 329, 5375 Ac St.
-95	Winifred	Hopper	Ap #140-8982 Velit Avenue
-97	Dana	Carter	814-601 Purus. Av.
-99	Wynter	Vincent	626-8492 Mollis Avenue
-92	Thane	Oneil	6766 Lectus St.
-94	Lael	Mclean	500-7010 Sit St.
-96	Rafael	England	P.O. Box 405, 7857 Eget Av.
-98	Juliet	Battle	Ap #535-1965 Cursus St.
-100	Wang	Mitchell	4023 Lacinia. Ave
diff --git a/sql/hive/src/test/resources/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 b/sql/hive/src/test/resources/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 b/sql/hive/src/test/resources/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 b/sql/hive/src/test/resources/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae b/sql/hive/src/test/resources/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 b/sql/hive/src/test/resources/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 b/sql/hive/src/test/resources/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 b/sql/hive/src/test/resources/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324
deleted file mode 100644
index ecd0704ce33862d684f8e71e56538bec82d719de..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-str                 	string              	None                
-mp                  	map<string,string>  	None                
-lst                 	array<string>       	None                
-strct               	struct<A:string,B:string>	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Fri Jan 31 19:56:26 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create_staging	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1391226986          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	colelction.delim    	,                   
-	field.delim         	|                   
-	mapkey.delim        	:                   
-	serialization.format	|                   
diff --git a/sql/hive/src/test/resources/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 b/sql/hive/src/test/resources/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187
deleted file mode 100644
index 6881640dd3cd44cb92e17a5388305a3b5e46c84b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	from deserializer   
-value               	string              	from deserializer   
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Fri Jan 31 19:56:27 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1391226987          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.ql.io.orc.OrcSerde	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.orc.OrcInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16
+++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d
index f799fca27c35305f23a617e37c5948a118834b02..5bba4beaf3355f54070bf03d3e2a068178d9b982 100644
--- a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d
+++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d
@@ -7,4 +7,4 @@
 255
 278
 98
-484
\ No newline at end of file
+484
diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-6ced6a6f5189c7a315d92ebf3dcc68d3
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a
rename to sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-6ced6a6f5189c7a315d92ebf3dcc68d3
diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9
+++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1
index f953881c894c1b8ba29a3d6f27e6daaa59830ff9..0b6d77d2e825b4b7d72ca661195340d33703e6bc 100644
--- a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1
+++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1
@@ -1 +1 @@
-1082202951192
\ No newline at end of file
+1082202951192
diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e b/sql/hive/src/test/resources/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657
deleted file mode 100644
index f172be3e72712a5ea4f1f65400102458d9b6979c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657
+++ /dev/null
@@ -1,10 +0,0 @@
-0	3	2
-165	NULL	1
-2	1	2
-238	NULL	1
-27	NULL	1
-311	NULL	1
-4	1	2
-5	3	2
-8	1	2
-86	NULL	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 b/sql/hive/src/test/resources/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf b/sql/hive/src/test/resources/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 b/sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333
deleted file mode 100644
index e3c0074c9d4f66508c7967395bf9f5460c39adb4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333
+++ /dev/null
@@ -1,9 +0,0 @@
-
-
-
-
-
-
-
-
-
diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 b/sql/hive/src/test/resources/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e b/sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e
deleted file mode 100644
index 4cd1242d9fe615027785ed5f8a7d9b21619cf806..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e
+++ /dev/null
@@ -1,10 +0,0 @@
-
-
-
-
-
-
-
-165
-255
-27
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f
deleted file mode 100644
index e7700b8aee5c692143aa65d406d64d963b52c14a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f
+++ /dev/null
@@ -1,5 +0,0 @@
-1
-NULL
-NULL
-NULL
-NULL
diff --git a/sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 b/sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9
index 217a1915f8826bc03b781c2c9a872348e628a87f..4dba25230ed1bf3c245268c4e12579e2dfa609e0 100644
--- a/sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9
+++ b/sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9
@@ -7,4 +7,4 @@
 5	val_5
 5	val_5
 8	val_8
-9	val_9
\ No newline at end of file
+9	val_9
diff --git a/sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 b/sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5
index ab34c87e42364b003835009d14d1419858b8d9f2..fde2ef3e83ce52d006895c27576f066d3e3547ad 100644
--- a/sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5
+++ b/sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5
@@ -7,4 +7,4 @@
 494	val_494
 493	val_493
 492	val_492
-492	val_492
\ No newline at end of file
+492	val_492
diff --git a/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f b/sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f
index 217a1915f8826bc03b781c2c9a872348e628a87f..4dba25230ed1bf3c245268c4e12579e2dfa609e0 100644
--- a/sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f
+++ b/sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f
@@ -7,4 +7,4 @@
 5	val_5
 5	val_5
 8	val_8
-9	val_9
\ No newline at end of file
+9	val_9
diff --git a/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 b/sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595
index 549da558b18da2990940d62d3f3e171cb6844748..83a26e92a694b713f51ec76d5d06ebcf9af73fbf 100644
--- a/sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595
+++ b/sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595
@@ -9,4 +9,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 19	val_19	19	val_19
-19	val_19	19	val_19
\ No newline at end of file
+19	val_19	19	val_19
diff --git a/sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 b/sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335
index 549da558b18da2990940d62d3f3e171cb6844748..83a26e92a694b713f51ec76d5d06ebcf9af73fbf 100644
--- a/sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335
+++ b/sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335
@@ -9,4 +9,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 19	val_19	19	val_19
-19	val_19	19	val_19
\ No newline at end of file
+19	val_19	19	val_19
diff --git a/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 b/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9
+++ b/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54 b/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54
+++ b/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a b/sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a
index c912af4a5c676a7d976d3d948451bd3a9670cd26..5710fb29e7e024f07f64557572bc25b60859a2d5 100644
--- a/sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a
+++ b/sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a
@@ -306,4 +306,4 @@
 495	val_495
 496	val_496
 497	val_497
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 b/sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43
index c912af4a5c676a7d976d3d948451bd3a9670cd26..5710fb29e7e024f07f64557572bc25b60859a2d5 100644
--- a/sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43
+++ b/sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43
@@ -306,4 +306,4 @@
 495	val_495
 496	val_496
 497	val_497
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a
+++ b/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a b/sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a
index c912af4a5c676a7d976d3d948451bd3a9670cd26..5710fb29e7e024f07f64557572bc25b60859a2d5 100644
--- a/sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a
+++ b/sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a
@@ -306,4 +306,4 @@
 495	val_495
 496	val_496
 497	val_497
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 b/sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43
index c912af4a5c676a7d976d3d948451bd3a9670cd26..5710fb29e7e024f07f64557572bc25b60859a2d5 100644
--- a/sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43
+++ b/sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43
@@ -306,4 +306,4 @@
 495	val_495
 496	val_496
 497	val_497
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c
+++ b/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d b/sql/hive/src/test/resources/golden/parallel_orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-1-be7423a5e77b7289153f74bc3fd97f44 b/sql/hive/src/test/resources/golden/parallel_orderby-1-be7423a5e77b7289153f74bc3fd97f44
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-11-39767502cbda75590c0c4b8fd089b793 b/sql/hive/src/test/resources/golden/parallel_orderby-11-39767502cbda75590c0c4b8fd089b793
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-12-7943485bcc72b2040c45c62e45ac8853 b/sql/hive/src/test/resources/golden/parallel_orderby-12-7943485bcc72b2040c45c62e45ac8853
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736 b/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736
deleted file mode 100644
index b431d3fc6dcf654dc67ffbd75dc6714768627625..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	None                
-value               	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Apr 29 20:55:07 PDT 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/total_ordered	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	1                   
-	numPartitions       	0                   
-	numRows             	48                  
-	rawDataSize         	512                 
-	totalSize           	560                 
-	transient_lastDdlTime	1398830107          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823 b/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823
deleted file mode 100644
index 3b733e2d6c4516183268d8ddaa5badc11ca58183..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823
+++ /dev/null
@@ -1,48 +0,0 @@
-128	val_128
-128	val_128
-150	val_150
-150	val_150
-165	val_165
-165	val_165
-193	val_193
-193	val_193
-213	val_213
-213	val_213
-213	val_213
-213	val_213
-213	val_214
-213	val_214
-224	val_224
-224	val_224
-238	val_238
-238	val_238
-238	val_239
-238	val_239
-238	val_240
-238	val_240
-255	val_255
-255	val_255
-265	val_265
-265	val_265
-27	val_27
-27	val_27
-273	val_273
-273	val_273
-278	val_278
-278	val_278
-311	val_311
-311	val_311
-369	val_369
-369	val_369
-401	val_401
-401	val_401
-409	val_409
-409	val_409
-484	val_484
-484	val_484
-66	val_66
-66	val_66
-86	val_86
-86	val_86
-98	val_98
-98	val_98
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-2-be7423a5e77b7289153f74bc3fd97f44 b/sql/hive/src/test/resources/golden/parallel_orderby-2-be7423a5e77b7289153f74bc3fd97f44
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-6-670ec1b1f28d92d72a924c29d622aa8f b/sql/hive/src/test/resources/golden/parallel_orderby-6-670ec1b1f28d92d72a924c29d622aa8f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-7-7943485bcc72b2040c45c62e45ac8853 b/sql/hive/src/test/resources/golden/parallel_orderby-7-7943485bcc72b2040c45c62e45ac8853
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736 b/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736
deleted file mode 100644
index 7f67251e617874fc891774132a15ad7c6e52774d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	None                
-value               	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Apr 29 20:54:55 PDT 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/total_ordered	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	1                   
-	numPartitions       	0                   
-	numRows             	48                  
-	rawDataSize         	512                 
-	totalSize           	560                 
-	transient_lastDdlTime	1398830095          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 b/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823
deleted file mode 100644
index 3b733e2d6c4516183268d8ddaa5badc11ca58183..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823
+++ /dev/null
@@ -1,48 +0,0 @@
-128	val_128
-128	val_128
-150	val_150
-150	val_150
-165	val_165
-165	val_165
-193	val_193
-193	val_193
-213	val_213
-213	val_213
-213	val_213
-213	val_213
-213	val_214
-213	val_214
-224	val_224
-224	val_224
-238	val_238
-238	val_238
-238	val_239
-238	val_239
-238	val_240
-238	val_240
-255	val_255
-255	val_255
-265	val_265
-265	val_265
-27	val_27
-27	val_27
-273	val_273
-273	val_273
-278	val_278
-278	val_278
-311	val_311
-311	val_311
-369	val_369
-369	val_369
-401	val_401
-401	val_401
-409	val_409
-409	val_409
-484	val_484
-484	val_484
-66	val_66
-66	val_66
-86	val_86
-86	val_86
-98	val_98
-98	val_98
diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 b/sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4
index 55d948599907211562da45bf68242aa0462d556f..b70e127e82d056a30f56d6141cafa419ffef0184 100644
--- a/sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4
+++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4
@@ -497,4 +497,4 @@
 497	val_497
 498	val_498
 498	val_498
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 b/sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44
index 55d948599907211562da45bf68242aa0462d556f..b70e127e82d056a30f56d6141cafa419ffef0184 100644
--- a/sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44
+++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44
@@ -497,4 +497,4 @@
 497	val_497
 498	val_498
 498	val_498
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc b/sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc
index 55d948599907211562da45bf68242aa0462d556f..b70e127e82d056a30f56d6141cafa419ffef0184 100644
--- a/sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc
+++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc
@@ -497,4 +497,4 @@
 497	val_497
 498	val_498
 498	val_498
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd b/sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd
index 55d948599907211562da45bf68242aa0462d556f..b70e127e82d056a30f56d6141cafa419ffef0184 100644
--- a/sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd
+++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd
@@ -497,4 +497,4 @@
 497	val_497
 498	val_498
 498	val_498
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec b/sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec
index 55d948599907211562da45bf68242aa0462d556f..b70e127e82d056a30f56d6141cafa419ffef0184 100644
--- a/sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec
+++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec
@@ -497,4 +497,4 @@
 497	val_497
 498	val_498
 498	val_498
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 b/sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189
index 55d948599907211562da45bf68242aa0462d556f..b70e127e82d056a30f56d6141cafa419ffef0184 100644
--- a/sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189
+++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189
@@ -497,4 +497,4 @@
 497	val_497
 498	val_498
 498	val_498
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 b/sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02
index 55d948599907211562da45bf68242aa0462d556f..b70e127e82d056a30f56d6141cafa419ffef0184 100644
--- a/sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02
+++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02
@@ -497,4 +497,4 @@
 497	val_497
 498	val_498
 498	val_498
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea b/sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea
index 55d948599907211562da45bf68242aa0462d556f..b70e127e82d056a30f56d6141cafa419ffef0184 100644
--- a/sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea
+++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea
@@ -497,4 +497,4 @@
 497	val_497
 498	val_498
 498	val_498
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 b/sql/hive/src/test/resources/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b b/sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 b/sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 b/sql/hive/src/test/resources/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 b/sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2
deleted file mode 100644
index 56a6051ca2b02b04ef92d5150c9ef600403cb1de..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2
+++ /dev/null
@@ -1 +0,0 @@
-1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0
+++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2
index ffa0de8a0c3221aa06102da474ccd2d24d73e224..a98dc8e8e1b86bd6f5e9a12785e53cd63f552cac 100644
--- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2
+++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2
@@ -1,24 +1,24 @@
 # col_name            	data_type           	comment             
 	 	 
-c1                  	tinyint             	None                
+c1                  	tinyint             	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-c2                  	string              	None                
+c2                  	string              	                    
 	 	 
 # Detailed Partition Information	 	 
 Partition Value:    	[v1]                	 
 Database:           	default             	 
 Table:              	mytbl               	 
-CreateTime:         	Fri Jan 03 17:18:52 PST 2014	 
+CreateTime:         	Tue Oct 21 04:00:26 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/mytbl/c2=v1	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/mytbl/c2=v1	 
 Partition Parameters:	 	 
 	a                   	myval               
 	b                   	yourval             
-	transient_lastDdlTime	1388798332          
+	transient_lastDdlTime	1413889226          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
@@ -29,4 +29,4 @@ Num Buckets:        	-1
 Bucket Columns:     	[]                  	 
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4
+++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7
+++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2
index 0909bab11a6ec01bf95e3b10cc5507cb17f7bb24..2bef99dafeb29b86cdedeb18421083307a3f6256 100644
--- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2
+++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2
@@ -1,22 +1,22 @@
 # col_name            	data_type           	comment             
 	 	 
-c1                  	tinyint             	None                
+c1                  	tinyint             	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-c2                  	string              	None                
+c2                  	string              	                    
 	 	 
 # Detailed Partition Information	 	 
 Partition Value:    	[v1]                	 
 Database:           	default             	 
 Table:              	mytbl               	 
-CreateTime:         	Fri Jan 03 17:18:59 PST 2014	 
+CreateTime:         	Tue Oct 21 04:00:27 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/mytbl/c2=v1	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/mytbl/c2=v1	 
 Partition Parameters:	 	 
-	transient_lastDdlTime	1388798339          
+	transient_lastDdlTime	1413889227          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
@@ -27,4 +27,4 @@ Num Buckets:        	-1
 Bucket Columns:     	[]                  	 
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06
+++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2
index 55fb6df62f0a76d93237cd9db8d87fa0c8401dfa..7b6b90f7f4200d4534228deeeb7c43185bf9e9bd 100644
--- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2
+++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2
@@ -1,25 +1,25 @@
 # col_name            	data_type           	comment             
 	 	 
-c1                  	tinyint             	None                
+c1                  	tinyint             	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-c2                  	string              	None                
+c2                  	string              	                    
 	 	 
 # Detailed Partition Information	 	 
 Partition Value:    	[v1]                	 
 Database:           	default             	 
 Table:              	mytbl               	 
-CreateTime:         	Fri Jan 03 17:19:02 PST 2014	 
+CreateTime:         	Tue Oct 21 04:00:28 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/mytbl/c2=v1	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/mytbl/c2=v1	 
 Partition Parameters:	 	 
 	a                   	myval               
 	b                   	yourval             
 	c                   	noval               
-	transient_lastDdlTime	1388798342          
+	transient_lastDdlTime	1413889228          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
@@ -30,4 +30,4 @@ Num Buckets:        	-1
 Bucket Columns:     	[]                  	 
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4
+++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 b/sql/hive/src/test/resources/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/partcols1-1-b562ff3e36de23a686d122967a1f91c8
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394
rename to sql/hive/src/test/resources/golden/partcols1-1-b562ff3e36de23a686d122967a1f91c8
diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/partition_date-1-6b0952309c3ebdd6dcb7066891d1bd74
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1
rename to sql/hive/src/test/resources/golden/partition_date-1-6b0952309c3ebdd6dcb7066891d1bd74
diff --git a/sql/hive/src/test/resources/golden/partition_date-10-aad6078a09b7bd8f5141437e86bb229f b/sql/hive/src/test/resources/golden/partition_date-10-aad6078a09b7bd8f5141437e86bb229f
new file mode 100644
index 0000000000000000000000000000000000000000..60d3b2f4a4cd5f1637eba020358bfe5ecb5edcf2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/partition_date-10-aad6078a09b7bd8f5141437e86bb229f
@@ -0,0 +1 @@
+15
diff --git a/sql/hive/src/test/resources/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 b/sql/hive/src/test/resources/golden/partition_date-11-a01c3791f59e819c750e213b6c65d084
similarity index 100%
rename from sql/hive/src/test/resources/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2
rename to sql/hive/src/test/resources/golden/partition_date-11-a01c3791f59e819c750e213b6c65d084
diff --git a/sql/hive/src/test/resources/golden/partition_date-12-2662a237c86cf9e4a150a4f1856b8239 b/sql/hive/src/test/resources/golden/partition_date-12-2662a237c86cf9e4a150a4f1856b8239
new file mode 100644
index 0000000000000000000000000000000000000000..b4de3947675361a7770d29b8982c407b0ec6b2a0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/partition_date-12-2662a237c86cf9e4a150a4f1856b8239
@@ -0,0 +1 @@
+11
diff --git a/sql/hive/src/test/resources/golden/partition_date-13-aa513c8ee1cbfd1c94f5772c110d4dc9 b/sql/hive/src/test/resources/golden/partition_date-13-aa513c8ee1cbfd1c94f5772c110d4dc9
new file mode 100644
index 0000000000000000000000000000000000000000..64bb6b746dceaf12b0ba8c08f310b0426babde44
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/partition_date-13-aa513c8ee1cbfd1c94f5772c110d4dc9
@@ -0,0 +1 @@
+30
diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/partition_date-14-3f187dad9a2fdfc6f7a3566309016f9c
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/partition_date-14-3f187dad9a2fdfc6f7a3566309016f9c
diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/partition_date-15-e4366325f3a0c4a8e92be59f4de73fce
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/partition_date-15-e4366325f3a0c4a8e92be59f4de73fce
diff --git a/sql/hive/src/test/resources/golden/partition_date-16-32cf81c1a36451eccb07b20dffd930ac b/sql/hive/src/test/resources/golden/partition_date-16-32cf81c1a36451eccb07b20dffd930ac
new file mode 100644
index 0000000000000000000000000000000000000000..209e3ef4b6247ce746048d5711befda46206d235
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/partition_date-16-32cf81c1a36451eccb07b20dffd930ac
@@ -0,0 +1 @@
+20
diff --git a/sql/hive/src/test/resources/golden/partition_date-17-8654528691598a5eef8e3c1059d24117 b/sql/hive/src/test/resources/golden/partition_date-17-8654528691598a5eef8e3c1059d24117
new file mode 100644
index 0000000000000000000000000000000000000000..f599e28b8ab0d8c9c57a486c89c4a5132dcbd3b2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/partition_date-17-8654528691598a5eef8e3c1059d24117
@@ -0,0 +1 @@
+10
diff --git a/sql/hive/src/test/resources/golden/partition_date-18-a1e769630ac3caed7325a3a256713b24 b/sql/hive/src/test/resources/golden/partition_date-18-a1e769630ac3caed7325a3a256713b24
new file mode 100644
index 0000000000000000000000000000000000000000..209e3ef4b6247ce746048d5711befda46206d235
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/partition_date-18-a1e769630ac3caed7325a3a256713b24
@@ -0,0 +1 @@
+20
diff --git a/sql/hive/src/test/resources/golden/partition_date-19-95f1399e12124840caee7492c0f3036d b/sql/hive/src/test/resources/golden/partition_date-19-95f1399e12124840caee7492c0f3036d
new file mode 100644
index 0000000000000000000000000000000000000000..f599e28b8ab0d8c9c57a486c89c4a5132dcbd3b2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/partition_date-19-95f1399e12124840caee7492c0f3036d
@@ -0,0 +1 @@
+10
diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/partition_date-2-220048240a7050a98411ddbc6b1f82cf
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/partition_date-2-220048240a7050a98411ddbc6b1f82cf
diff --git a/sql/hive/src/test/resources/golden/partition_date-20-e734eb4fc8894c9a2b2b9cdac4270fba b/sql/hive/src/test/resources/golden/partition_date-20-e734eb4fc8894c9a2b2b9cdac4270fba
new file mode 100644
index 0000000000000000000000000000000000000000..209e3ef4b6247ce746048d5711befda46206d235
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/partition_date-20-e734eb4fc8894c9a2b2b9cdac4270fba
@@ -0,0 +1 @@
+20
diff --git a/sql/hive/src/test/resources/golden/partition_date-21-b60c0a3677591991152a0aa500bdce68 b/sql/hive/src/test/resources/golden/partition_date-21-b60c0a3677591991152a0aa500bdce68
new file mode 100644
index 0000000000000000000000000000000000000000..f599e28b8ab0d8c9c57a486c89c4a5132dcbd3b2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/partition_date-21-b60c0a3677591991152a0aa500bdce68
@@ -0,0 +1 @@
+10
diff --git a/sql/hive/src/test/resources/golden/type_cast_1-1-53a667981ad567b2ab977f67d65c5825 b/sql/hive/src/test/resources/golden/partition_date-22-fe277bd0a30e016826d242fd5bd61714
similarity index 100%
rename from sql/hive/src/test/resources/golden/type_cast_1-1-53a667981ad567b2ab977f67d65c5825
rename to sql/hive/src/test/resources/golden/partition_date-22-fe277bd0a30e016826d242fd5bd61714
diff --git a/sql/hive/src/test/resources/golden/partition_date-23-7f014b494185e4c2a6048efb89131344 b/sql/hive/src/test/resources/golden/partition_date-23-7f014b494185e4c2a6048efb89131344
new file mode 100644
index 0000000000000000000000000000000000000000..7ed6ff82de6bcc2a78243fc9c54d3ef5ac14da69
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/partition_date-23-7f014b494185e4c2a6048efb89131344
@@ -0,0 +1 @@
+5
diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/partition_date-24-7ec1f3a845e2c49191460e15af30aa30
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc
rename to sql/hive/src/test/resources/golden/partition_date-24-7ec1f3a845e2c49191460e15af30aa30
diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/partition_date-3-f8887dd18c21bf0306e293d463b3e1d7
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a
rename to sql/hive/src/test/resources/golden/partition_date-3-f8887dd18c21bf0306e293d463b3e1d7
diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/partition_date-4-f7c20c208e052305c1710a5e1e80c5c8
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394
rename to sql/hive/src/test/resources/golden/partition_date-4-f7c20c208e052305c1710a5e1e80c5c8
diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/partition_date-5-59355f4e222dcd4f77f51d15bd896f11
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1
rename to sql/hive/src/test/resources/golden/partition_date-5-59355f4e222dcd4f77f51d15bd896f11
diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/partition_date-6-416ab10ac818d432cd3a81870d6e5164
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/partition_date-6-416ab10ac818d432cd3a81870d6e5164
diff --git a/sql/hive/src/test/resources/golden/partition_date-7-1405c311915f27b0cc616c83d39eaacc b/sql/hive/src/test/resources/golden/partition_date-7-1405c311915f27b0cc616c83d39eaacc
new file mode 100644
index 0000000000000000000000000000000000000000..758a01e98747380d5b34df217af985ec4370c377
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/partition_date-7-1405c311915f27b0cc616c83d39eaacc
@@ -0,0 +1,3 @@
+2000-01-01
+2013-08-08
+2013-12-10
diff --git a/sql/hive/src/test/resources/golden/partition_date-8-7703adfcfdd44c9250f9eba478004714 b/sql/hive/src/test/resources/golden/partition_date-8-7703adfcfdd44c9250f9eba478004714
new file mode 100644
index 0000000000000000000000000000000000000000..24192eefd2cafb2dd8d560e8f43f4e5b2389a44f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/partition_date-8-7703adfcfdd44c9250f9eba478004714
@@ -0,0 +1,5 @@
+165	val_165	2000-01-01	2
+238	val_238	2000-01-01	2
+27	val_27	2000-01-01	2
+311	val_311	2000-01-01	2
+86	val_86	2000-01-01	2
diff --git a/sql/hive/src/test/resources/golden/partition_date-9-a425c11c12c9ce4c9c43d4fbccee5347 b/sql/hive/src/test/resources/golden/partition_date-9-a425c11c12c9ce4c9c43d4fbccee5347
new file mode 100644
index 0000000000000000000000000000000000000000..60d3b2f4a4cd5f1637eba020358bfe5ecb5edcf2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/partition_date-9-a425c11c12c9ce4c9c43d4fbccee5347
@@ -0,0 +1 @@
+15
diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 b/sql/hive/src/test/resources/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 b/sql/hive/src/test/resources/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5
deleted file mode 100644
index 81df1796184064c85116f253f9f034ad5eacb4f8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5
+++ /dev/null
@@ -1,3 +0,0 @@
-ts=2011-01-11+14:18:26
-ts=2011-01-11+15:18:26
-ts=2011-01-11+16:18:26
diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618
deleted file mode 100644
index 00750edc07d6415dcc07ae0351e9397b0222b7ba..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618
+++ /dev/null
@@ -1 +0,0 @@
-3
diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5
deleted file mode 100644
index e16053e3110aedf5b42a581a5a6c4f50ce4f3079..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5
+++ /dev/null
@@ -1,3 +0,0 @@
-ts=2011-01-11+14%3A18%3A26
-ts=2011-01-11+15%3A18%3A26
-ts=2011-01-11+16%3A18%3A26
diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618
deleted file mode 100644
index 00750edc07d6415dcc07ae0351e9397b0222b7ba..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618
+++ /dev/null
@@ -1 +0,0 @@
-3
diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 b/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988
index 8af58e2b29f4a56b39b2f93066a4df9702b3166e..c97e50a8a58cdd53e9ad6bbdee812c668a666477 100644
--- a/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988
+++ b/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988
@@ -1,8 +1,8 @@
-key                 	string              	None                
-value               	string              	None                
-dt                  	string              	None                
+key                 	string              	                    
+value               	string              	                    
+dt                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-dt                  	string              	None                
\ No newline at end of file
+dt                  	string              	                    
diff --git a/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 b/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7
index 82a07522b0af03433d83583ab413192982f94b60..39db984c884ad53efffab2c7bc7f9b879b3d651d 100644
--- a/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7
+++ b/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7
@@ -1,9 +1,9 @@
-key                 	string              	None                
-value               	string              	None                
-x                   	string              	None                
-dt                  	string              	None                
+key                 	string              	                    
+value               	string              	                    
+x                   	string              	                    
+dt                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-dt                  	string              	None                
\ No newline at end of file
+dt                  	string              	                    
diff --git a/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 b/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634
index 8af58e2b29f4a56b39b2f93066a4df9702b3166e..c97e50a8a58cdd53e9ad6bbdee812c668a666477 100644
--- a/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634
+++ b/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634
@@ -1,8 +1,8 @@
-key                 	string              	None                
-value               	string              	None                
-dt                  	string              	None                
+key                 	string              	                    
+value               	string              	                    
+dt                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-dt                  	string              	None                
\ No newline at end of file
+dt                  	string              	                    
diff --git a/sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 b/sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4
index 95f5492558a9bf6c389a1ca15c028fd77ab0a06c..7e53c64af58aa0dedd16d783d4e173f1c6d932f6 100644
--- a/sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4
+++ b/sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4
@@ -17,4 +17,4 @@
 17	val_17
 18	val_18
 18	val_18
-19	val_19
\ No newline at end of file
+19	val_19
diff --git a/sql/hive/src/test/resources/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 b/sql/hive/src/test/resources/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 b/sql/hive/src/test/resources/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5
deleted file mode 100644
index e16053e3110aedf5b42a581a5a6c4f50ce4f3079..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5
+++ /dev/null
@@ -1,3 +0,0 @@
-ts=2011-01-11+14%3A18%3A26
-ts=2011-01-11+15%3A18%3A26
-ts=2011-01-11+16%3A18%3A26
diff --git a/sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618
deleted file mode 100644
index 00750edc07d6415dcc07ae0351e9397b0222b7ba..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618
+++ /dev/null
@@ -1 +0,0 @@
-3
diff --git a/sql/hive/src/test/resources/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5
deleted file mode 100644
index e16053e3110aedf5b42a581a5a6c4f50ce4f3079..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5
+++ /dev/null
@@ -1,3 +0,0 @@
-ts=2011-01-11+14%3A18%3A26
-ts=2011-01-11+15%3A18%3A26
-ts=2011-01-11+16%3A18%3A26
diff --git a/sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618
deleted file mode 100644
index 00750edc07d6415dcc07ae0351e9397b0222b7ba..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618
+++ /dev/null
@@ -1 +0,0 @@
-3
diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/partition_type_check-11-60aad884be613d18d3b89fca3b90dc94
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc
rename to sql/hive/src/test/resources/golden/partition_type_check-11-60aad884be613d18d3b89fca3b90dc94
diff --git a/sql/hive/src/test/resources/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a b/sql/hive/src/test/resources/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/partition_type_check-2-3a6bb204d9524299f28adf1cc35d6f4d
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a
rename to sql/hive/src/test/resources/golden/partition_type_check-2-3a6bb204d9524299f28adf1cc35d6f4d
diff --git a/sql/hive/src/test/resources/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f b/sql/hive/src/test/resources/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 b/sql/hive/src/test/resources/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/partition_type_check-6-663051c7106b8ee6913ca1b007157941
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394
rename to sql/hive/src/test/resources/golden/partition_type_check-6-663051c7106b8ee6913ca1b007157941
diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/partition_varchar1-2-bca5c9edccc3a84e0a9ef92ebcbe746
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1
rename to sql/hive/src/test/resources/golden/partition_varchar1-2-bca5c9edccc3a84e0a9ef92ebcbe746
diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 b/sql/hive/src/test/resources/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 b/sql/hive/src/test/resources/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/partition_varchar1-3-9cec4b1d156f5a9cb587470b98de15
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/partition_varchar1-3-9cec4b1d156f5a9cb587470b98de15
diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef b/sql/hive/src/test/resources/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/partition_varchar1-4-e9c4530e270db6d44cc54292e4eff680
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc
rename to sql/hive/src/test/resources/golden/partition_varchar1-4-e9c4530e270db6d44cc54292e4eff680
diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/partition_varchar1-5-661aefd18c44c1eb02c2aaf8fab59f73
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a
rename to sql/hive/src/test/resources/golden/partition_varchar1-5-661aefd18c44c1eb02c2aaf8fab59f73
diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce b/sql/hive/src/test/resources/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde
deleted file mode 100644
index 39c80f1b77eab16fb1689b825ace80f4161d5061..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde
+++ /dev/null
@@ -1,1000 +0,0 @@
-0	val_0	NULL
-0	val_0	NULL
-0	val_0	NULL
-0	val_0	0
-0	val_0	0
-0	val_0	0
-10	val_10	NULL
-10	val_10	10
-100	val_100	NULL
-100	val_100	NULL
-100	val_100	100
-100	val_100	100
-103	val_103	NULL
-103	val_103	NULL
-103	val_103	103
-103	val_103	103
-104	val_104	NULL
-104	val_104	NULL
-104	val_104	104
-104	val_104	104
-105	val_105	NULL
-105	val_105	105
-11	val_11	NULL
-11	val_11	11
-111	val_111	NULL
-111	val_111	111
-113	val_113	NULL
-113	val_113	NULL
-113	val_113	113
-113	val_113	113
-114	val_114	NULL
-114	val_114	114
-116	val_116	NULL
-116	val_116	116
-118	val_118	NULL
-118	val_118	NULL
-118	val_118	118
-118	val_118	118
-119	val_119	NULL
-119	val_119	NULL
-119	val_119	NULL
-119	val_119	119
-119	val_119	119
-119	val_119	119
-12	val_12	NULL
-12	val_12	NULL
-12	val_12	12
-12	val_12	12
-120	val_120	NULL
-120	val_120	NULL
-120	val_120	120
-120	val_120	120
-125	val_125	NULL
-125	val_125	NULL
-125	val_125	125
-125	val_125	125
-126	val_126	NULL
-126	val_126	126
-128	val_128	NULL
-128	val_128	NULL
-128	val_128	NULL
-128	val_128	128
-128	val_128	128
-128	val_128	128
-129	val_129	NULL
-129	val_129	NULL
-129	val_129	129
-129	val_129	129
-131	val_131	NULL
-131	val_131	131
-133	val_133	NULL
-133	val_133	133
-134	val_134	NULL
-134	val_134	NULL
-134	val_134	134
-134	val_134	134
-136	val_136	NULL
-136	val_136	136
-137	val_137	NULL
-137	val_137	NULL
-137	val_137	137
-137	val_137	137
-138	val_138	NULL
-138	val_138	NULL
-138	val_138	NULL
-138	val_138	NULL
-138	val_138	138
-138	val_138	138
-138	val_138	138
-138	val_138	138
-143	val_143	NULL
-143	val_143	143
-145	val_145	NULL
-145	val_145	145
-146	val_146	NULL
-146	val_146	NULL
-146	val_146	146
-146	val_146	146
-149	val_149	NULL
-149	val_149	NULL
-149	val_149	149
-149	val_149	149
-15	val_15	NULL
-15	val_15	NULL
-15	val_15	15
-15	val_15	15
-150	val_150	NULL
-150	val_150	150
-152	val_152	NULL
-152	val_152	NULL
-152	val_152	152
-152	val_152	152
-153	val_153	NULL
-153	val_153	153
-155	val_155	NULL
-155	val_155	155
-156	val_156	NULL
-156	val_156	156
-157	val_157	NULL
-157	val_157	157
-158	val_158	NULL
-158	val_158	158
-160	val_160	NULL
-160	val_160	160
-162	val_162	NULL
-162	val_162	162
-163	val_163	NULL
-163	val_163	163
-164	val_164	NULL
-164	val_164	NULL
-164	val_164	164
-164	val_164	164
-165	val_165	NULL
-165	val_165	NULL
-165	val_165	165
-165	val_165	165
-166	val_166	NULL
-166	val_166	166
-167	val_167	NULL
-167	val_167	NULL
-167	val_167	NULL
-167	val_167	167
-167	val_167	167
-167	val_167	167
-168	val_168	NULL
-168	val_168	168
-169	val_169	NULL
-169	val_169	NULL
-169	val_169	NULL
-169	val_169	NULL
-169	val_169	169
-169	val_169	169
-169	val_169	169
-169	val_169	169
-17	val_17	NULL
-17	val_17	17
-170	val_170	NULL
-170	val_170	170
-172	val_172	NULL
-172	val_172	NULL
-172	val_172	172
-172	val_172	172
-174	val_174	NULL
-174	val_174	NULL
-174	val_174	174
-174	val_174	174
-175	val_175	NULL
-175	val_175	NULL
-175	val_175	175
-175	val_175	175
-176	val_176	NULL
-176	val_176	NULL
-176	val_176	176
-176	val_176	176
-177	val_177	NULL
-177	val_177	177
-178	val_178	NULL
-178	val_178	178
-179	val_179	NULL
-179	val_179	NULL
-179	val_179	179
-179	val_179	179
-18	val_18	NULL
-18	val_18	NULL
-18	val_18	18
-18	val_18	18
-180	val_180	NULL
-180	val_180	180
-181	val_181	NULL
-181	val_181	181
-183	val_183	NULL
-183	val_183	183
-186	val_186	NULL
-186	val_186	186
-187	val_187	NULL
-187	val_187	NULL
-187	val_187	NULL
-187	val_187	187
-187	val_187	187
-187	val_187	187
-189	val_189	NULL
-189	val_189	189
-19	val_19	NULL
-19	val_19	19
-190	val_190	NULL
-190	val_190	190
-191	val_191	NULL
-191	val_191	NULL
-191	val_191	191
-191	val_191	191
-192	val_192	NULL
-192	val_192	192
-193	val_193	NULL
-193	val_193	NULL
-193	val_193	NULL
-193	val_193	193
-193	val_193	193
-193	val_193	193
-194	val_194	NULL
-194	val_194	194
-195	val_195	NULL
-195	val_195	NULL
-195	val_195	195
-195	val_195	195
-196	val_196	NULL
-196	val_196	196
-197	val_197	NULL
-197	val_197	NULL
-197	val_197	197
-197	val_197	197
-199	val_199	NULL
-199	val_199	NULL
-199	val_199	NULL
-199	val_199	199
-199	val_199	199
-199	val_199	199
-2	val_2	NULL
-2	val_2	2
-20	val_20	NULL
-20	val_20	20
-200	val_200	NULL
-200	val_200	NULL
-200	val_200	200
-200	val_200	200
-201	val_201	NULL
-201	val_201	201
-202	val_202	NULL
-202	val_202	202
-203	val_203	NULL
-203	val_203	NULL
-203	val_203	203
-203	val_203	203
-205	val_205	NULL
-205	val_205	NULL
-205	val_205	205
-205	val_205	205
-207	val_207	NULL
-207	val_207	NULL
-207	val_207	207
-207	val_207	207
-208	val_208	NULL
-208	val_208	NULL
-208	val_208	NULL
-208	val_208	208
-208	val_208	208
-208	val_208	208
-209	val_209	NULL
-209	val_209	NULL
-209	val_209	209
-209	val_209	209
-213	val_213	NULL
-213	val_213	NULL
-213	val_213	213
-213	val_213	213
-214	val_214	NULL
-214	val_214	214
-216	val_216	NULL
-216	val_216	NULL
-216	val_216	216
-216	val_216	216
-217	val_217	NULL
-217	val_217	NULL
-217	val_217	217
-217	val_217	217
-218	val_218	NULL
-218	val_218	218
-219	val_219	NULL
-219	val_219	NULL
-219	val_219	219
-219	val_219	219
-221	val_221	NULL
-221	val_221	NULL
-221	val_221	221
-221	val_221	221
-222	val_222	NULL
-222	val_222	222
-223	val_223	NULL
-223	val_223	NULL
-223	val_223	223
-223	val_223	223
-224	val_224	NULL
-224	val_224	NULL
-224	val_224	224
-224	val_224	224
-226	val_226	NULL
-226	val_226	226
-228	val_228	NULL
-228	val_228	228
-229	val_229	NULL
-229	val_229	NULL
-229	val_229	229
-229	val_229	229
-230	val_230	NULL
-230	val_230	NULL
-230	val_230	NULL
-230	val_230	NULL
-230	val_230	NULL
-230	val_230	230
-230	val_230	230
-230	val_230	230
-230	val_230	230
-230	val_230	230
-233	val_233	NULL
-233	val_233	NULL
-233	val_233	233
-233	val_233	233
-235	val_235	NULL
-235	val_235	235
-237	val_237	NULL
-237	val_237	NULL
-237	val_237	237
-237	val_237	237
-238	val_238	NULL
-238	val_238	NULL
-238	val_238	238
-238	val_238	238
-239	val_239	NULL
-239	val_239	NULL
-239	val_239	239
-239	val_239	239
-24	val_24	NULL
-24	val_24	NULL
-24	val_24	24
-24	val_24	24
-241	val_241	NULL
-241	val_241	241
-242	val_242	NULL
-242	val_242	NULL
-242	val_242	242
-242	val_242	242
-244	val_244	NULL
-244	val_244	244
-247	val_247	NULL
-247	val_247	247
-248	val_248	NULL
-248	val_248	248
-249	val_249	NULL
-249	val_249	249
-252	val_252	NULL
-252	val_252	252
-255	val_255	NULL
-255	val_255	NULL
-255	val_255	255
-255	val_255	255
-256	val_256	NULL
-256	val_256	NULL
-256	val_256	256
-256	val_256	256
-257	val_257	NULL
-257	val_257	257
-258	val_258	NULL
-258	val_258	258
-26	val_26	NULL
-26	val_26	NULL
-26	val_26	26
-26	val_26	26
-260	val_260	NULL
-260	val_260	260
-262	val_262	NULL
-262	val_262	262
-263	val_263	NULL
-263	val_263	263
-265	val_265	NULL
-265	val_265	NULL
-265	val_265	265
-265	val_265	265
-266	val_266	NULL
-266	val_266	266
-27	val_27	NULL
-27	val_27	27
-272	val_272	NULL
-272	val_272	NULL
-272	val_272	272
-272	val_272	272
-273	val_273	NULL
-273	val_273	NULL
-273	val_273	NULL
-273	val_273	273
-273	val_273	273
-273	val_273	273
-274	val_274	NULL
-274	val_274	274
-275	val_275	NULL
-275	val_275	275
-277	val_277	NULL
-277	val_277	NULL
-277	val_277	NULL
-277	val_277	NULL
-277	val_277	277
-277	val_277	277
-277	val_277	277
-277	val_277	277
-278	val_278	NULL
-278	val_278	NULL
-278	val_278	278
-278	val_278	278
-28	val_28	NULL
-28	val_28	28
-280	val_280	NULL
-280	val_280	NULL
-280	val_280	280
-280	val_280	280
-281	val_281	NULL
-281	val_281	NULL
-281	val_281	281
-281	val_281	281
-282	val_282	NULL
-282	val_282	NULL
-282	val_282	282
-282	val_282	282
-283	val_283	NULL
-283	val_283	283
-284	val_284	NULL
-284	val_284	284
-285	val_285	NULL
-285	val_285	285
-286	val_286	NULL
-286	val_286	286
-287	val_287	NULL
-287	val_287	287
-288	val_288	NULL
-288	val_288	NULL
-288	val_288	288
-288	val_288	288
-289	val_289	NULL
-289	val_289	289
-291	val_291	NULL
-291	val_291	291
-292	val_292	NULL
-292	val_292	292
-296	val_296	NULL
-296	val_296	296
-298	val_298	NULL
-298	val_298	NULL
-298	val_298	NULL
-298	val_298	298
-298	val_298	298
-298	val_298	298
-30	val_30	NULL
-30	val_30	30
-302	val_302	NULL
-302	val_302	302
-305	val_305	NULL
-305	val_305	305
-306	val_306	NULL
-306	val_306	306
-307	val_307	NULL
-307	val_307	NULL
-307	val_307	307
-307	val_307	307
-308	val_308	NULL
-308	val_308	308
-309	val_309	NULL
-309	val_309	NULL
-309	val_309	309
-309	val_309	309
-310	val_310	NULL
-310	val_310	310
-311	val_311	NULL
-311	val_311	NULL
-311	val_311	NULL
-311	val_311	311
-311	val_311	311
-311	val_311	311
-315	val_315	NULL
-315	val_315	315
-316	val_316	NULL
-316	val_316	NULL
-316	val_316	NULL
-316	val_316	316
-316	val_316	316
-316	val_316	316
-317	val_317	NULL
-317	val_317	NULL
-317	val_317	317
-317	val_317	317
-318	val_318	NULL
-318	val_318	NULL
-318	val_318	NULL
-318	val_318	318
-318	val_318	318
-318	val_318	318
-321	val_321	NULL
-321	val_321	NULL
-321	val_321	321
-321	val_321	321
-322	val_322	NULL
-322	val_322	NULL
-322	val_322	322
-322	val_322	322
-323	val_323	NULL
-323	val_323	323
-325	val_325	NULL
-325	val_325	NULL
-325	val_325	325
-325	val_325	325
-327	val_327	NULL
-327	val_327	NULL
-327	val_327	NULL
-327	val_327	327
-327	val_327	327
-327	val_327	327
-33	val_33	NULL
-33	val_33	33
-331	val_331	NULL
-331	val_331	NULL
-331	val_331	331
-331	val_331	331
-332	val_332	NULL
-332	val_332	332
-333	val_333	NULL
-333	val_333	NULL
-333	val_333	333
-333	val_333	333
-335	val_335	NULL
-335	val_335	335
-336	val_336	NULL
-336	val_336	336
-338	val_338	NULL
-338	val_338	338
-339	val_339	NULL
-339	val_339	339
-34	val_34	NULL
-34	val_34	34
-341	val_341	NULL
-341	val_341	341
-342	val_342	NULL
-342	val_342	NULL
-342	val_342	342
-342	val_342	342
-344	val_344	NULL
-344	val_344	NULL
-344	val_344	344
-344	val_344	344
-345	val_345	NULL
-345	val_345	345
-348	val_348	NULL
-348	val_348	NULL
-348	val_348	NULL
-348	val_348	NULL
-348	val_348	NULL
-348	val_348	348
-348	val_348	348
-348	val_348	348
-348	val_348	348
-348	val_348	348
-35	val_35	NULL
-35	val_35	NULL
-35	val_35	NULL
-35	val_35	35
-35	val_35	35
-35	val_35	35
-351	val_351	NULL
-351	val_351	351
-353	val_353	NULL
-353	val_353	NULL
-353	val_353	353
-353	val_353	353
-356	val_356	NULL
-356	val_356	356
-360	val_360	NULL
-360	val_360	360
-362	val_362	NULL
-362	val_362	362
-364	val_364	NULL
-364	val_364	364
-365	val_365	NULL
-365	val_365	365
-366	val_366	NULL
-366	val_366	366
-367	val_367	NULL
-367	val_367	NULL
-367	val_367	367
-367	val_367	367
-368	val_368	NULL
-368	val_368	368
-369	val_369	NULL
-369	val_369	NULL
-369	val_369	NULL
-369	val_369	369
-369	val_369	369
-369	val_369	369
-37	val_37	NULL
-37	val_37	NULL
-37	val_37	37
-37	val_37	37
-373	val_373	NULL
-373	val_373	373
-374	val_374	NULL
-374	val_374	374
-375	val_375	NULL
-375	val_375	375
-377	val_377	NULL
-377	val_377	377
-378	val_378	NULL
-378	val_378	378
-379	val_379	NULL
-379	val_379	379
-382	val_382	NULL
-382	val_382	NULL
-382	val_382	382
-382	val_382	382
-384	val_384	NULL
-384	val_384	NULL
-384	val_384	NULL
-384	val_384	384
-384	val_384	384
-384	val_384	384
-386	val_386	NULL
-386	val_386	386
-389	val_389	NULL
-389	val_389	389
-392	val_392	NULL
-392	val_392	392
-393	val_393	NULL
-393	val_393	393
-394	val_394	NULL
-394	val_394	394
-395	val_395	NULL
-395	val_395	NULL
-395	val_395	395
-395	val_395	395
-396	val_396	NULL
-396	val_396	NULL
-396	val_396	NULL
-396	val_396	396
-396	val_396	396
-396	val_396	396
-397	val_397	NULL
-397	val_397	NULL
-397	val_397	397
-397	val_397	397
-399	val_399	NULL
-399	val_399	NULL
-399	val_399	399
-399	val_399	399
-4	val_4	NULL
-4	val_4	4
-400	val_400	NULL
-400	val_400	400
-401	val_401	NULL
-401	val_401	NULL
-401	val_401	NULL
-401	val_401	NULL
-401	val_401	NULL
-401	val_401	401
-401	val_401	401
-401	val_401	401
-401	val_401	401
-401	val_401	401
-402	val_402	NULL
-402	val_402	402
-403	val_403	NULL
-403	val_403	NULL
-403	val_403	NULL
-403	val_403	403
-403	val_403	403
-403	val_403	403
-404	val_404	NULL
-404	val_404	NULL
-404	val_404	404
-404	val_404	404
-406	val_406	NULL
-406	val_406	NULL
-406	val_406	NULL
-406	val_406	NULL
-406	val_406	406
-406	val_406	406
-406	val_406	406
-406	val_406	406
-407	val_407	NULL
-407	val_407	407
-409	val_409	NULL
-409	val_409	NULL
-409	val_409	NULL
-409	val_409	409
-409	val_409	409
-409	val_409	409
-41	val_41	NULL
-41	val_41	41
-411	val_411	NULL
-411	val_411	411
-413	val_413	NULL
-413	val_413	NULL
-413	val_413	413
-413	val_413	413
-414	val_414	NULL
-414	val_414	NULL
-414	val_414	414
-414	val_414	414
-417	val_417	NULL
-417	val_417	NULL
-417	val_417	NULL
-417	val_417	417
-417	val_417	417
-417	val_417	417
-418	val_418	NULL
-418	val_418	418
-419	val_419	NULL
-419	val_419	419
-42	val_42	NULL
-42	val_42	NULL
-42	val_42	42
-42	val_42	42
-421	val_421	NULL
-421	val_421	421
-424	val_424	NULL
-424	val_424	NULL
-424	val_424	424
-424	val_424	424
-427	val_427	NULL
-427	val_427	427
-429	val_429	NULL
-429	val_429	NULL
-429	val_429	429
-429	val_429	429
-43	val_43	NULL
-43	val_43	43
-430	val_430	NULL
-430	val_430	NULL
-430	val_430	NULL
-430	val_430	430
-430	val_430	430
-430	val_430	430
-431	val_431	NULL
-431	val_431	NULL
-431	val_431	NULL
-431	val_431	431
-431	val_431	431
-431	val_431	431
-432	val_432	NULL
-432	val_432	432
-435	val_435	NULL
-435	val_435	435
-436	val_436	NULL
-436	val_436	436
-437	val_437	NULL
-437	val_437	437
-438	val_438	NULL
-438	val_438	NULL
-438	val_438	NULL
-438	val_438	438
-438	val_438	438
-438	val_438	438
-439	val_439	NULL
-439	val_439	NULL
-439	val_439	439
-439	val_439	439
-44	val_44	NULL
-44	val_44	44
-443	val_443	NULL
-443	val_443	443
-444	val_444	NULL
-444	val_444	444
-446	val_446	NULL
-446	val_446	446
-448	val_448	NULL
-448	val_448	448
-449	val_449	NULL
-449	val_449	449
-452	val_452	NULL
-452	val_452	452
-453	val_453	NULL
-453	val_453	453
-454	val_454	NULL
-454	val_454	NULL
-454	val_454	NULL
-454	val_454	454
-454	val_454	454
-454	val_454	454
-455	val_455	NULL
-455	val_455	455
-457	val_457	NULL
-457	val_457	457
-458	val_458	NULL
-458	val_458	NULL
-458	val_458	458
-458	val_458	458
-459	val_459	NULL
-459	val_459	NULL
-459	val_459	459
-459	val_459	459
-460	val_460	NULL
-460	val_460	460
-462	val_462	NULL
-462	val_462	NULL
-462	val_462	462
-462	val_462	462
-463	val_463	NULL
-463	val_463	NULL
-463	val_463	463
-463	val_463	463
-466	val_466	NULL
-466	val_466	NULL
-466	val_466	NULL
-466	val_466	466
-466	val_466	466
-466	val_466	466
-467	val_467	NULL
-467	val_467	467
-468	val_468	NULL
-468	val_468	NULL
-468	val_468	NULL
-468	val_468	NULL
-468	val_468	468
-468	val_468	468
-468	val_468	468
-468	val_468	468
-469	val_469	NULL
-469	val_469	NULL
-469	val_469	NULL
-469	val_469	NULL
-469	val_469	NULL
-469	val_469	469
-469	val_469	469
-469	val_469	469
-469	val_469	469
-469	val_469	469
-47	val_47	NULL
-47	val_47	47
-470	val_470	NULL
-470	val_470	470
-472	val_472	NULL
-472	val_472	472
-475	val_475	NULL
-475	val_475	475
-477	val_477	NULL
-477	val_477	477
-478	val_478	NULL
-478	val_478	NULL
-478	val_478	478
-478	val_478	478
-479	val_479	NULL
-479	val_479	479
-480	val_480	NULL
-480	val_480	NULL
-480	val_480	NULL
-480	val_480	480
-480	val_480	480
-480	val_480	480
-481	val_481	NULL
-481	val_481	481
-482	val_482	NULL
-482	val_482	482
-483	val_483	NULL
-483	val_483	483
-484	val_484	NULL
-484	val_484	484
-485	val_485	NULL
-485	val_485	485
-487	val_487	NULL
-487	val_487	487
-489	val_489	NULL
-489	val_489	NULL
-489	val_489	NULL
-489	val_489	NULL
-489	val_489	489
-489	val_489	489
-489	val_489	489
-489	val_489	489
-490	val_490	NULL
-490	val_490	490
-491	val_491	NULL
-491	val_491	491
-492	val_492	NULL
-492	val_492	NULL
-492	val_492	492
-492	val_492	492
-493	val_493	NULL
-493	val_493	493
-494	val_494	NULL
-494	val_494	494
-495	val_495	NULL
-495	val_495	495
-496	val_496	NULL
-496	val_496	496
-497	val_497	NULL
-497	val_497	497
-498	val_498	NULL
-498	val_498	NULL
-498	val_498	NULL
-498	val_498	498
-498	val_498	498
-498	val_498	498
-5	val_5	NULL
-5	val_5	NULL
-5	val_5	NULL
-5	val_5	5
-5	val_5	5
-5	val_5	5
-51	val_51	NULL
-51	val_51	NULL
-51	val_51	51
-51	val_51	51
-53	val_53	NULL
-53	val_53	53
-54	val_54	NULL
-54	val_54	54
-57	val_57	NULL
-57	val_57	57
-58	val_58	NULL
-58	val_58	NULL
-58	val_58	58
-58	val_58	58
-64	val_64	NULL
-64	val_64	64
-65	val_65	NULL
-65	val_65	65
-66	val_66	NULL
-66	val_66	66
-67	val_67	NULL
-67	val_67	NULL
-67	val_67	67
-67	val_67	67
-69	val_69	NULL
-69	val_69	69
-70	val_70	NULL
-70	val_70	NULL
-70	val_70	NULL
-70	val_70	70
-70	val_70	70
-70	val_70	70
-72	val_72	NULL
-72	val_72	NULL
-72	val_72	72
-72	val_72	72
-74	val_74	NULL
-74	val_74	74
-76	val_76	NULL
-76	val_76	NULL
-76	val_76	76
-76	val_76	76
-77	val_77	NULL
-77	val_77	77
-78	val_78	NULL
-78	val_78	78
-8	val_8	NULL
-8	val_8	8
-80	val_80	NULL
-80	val_80	80
-82	val_82	NULL
-82	val_82	82
-83	val_83	NULL
-83	val_83	NULL
-83	val_83	83
-83	val_83	83
-84	val_84	NULL
-84	val_84	NULL
-84	val_84	84
-84	val_84	84
-85	val_85	NULL
-85	val_85	85
-86	val_86	NULL
-86	val_86	86
-87	val_87	NULL
-87	val_87	87
-9	val_9	NULL
-9	val_9	9
-90	val_90	NULL
-90	val_90	NULL
-90	val_90	NULL
-90	val_90	90
-90	val_90	90
-90	val_90	90
-92	val_92	NULL
-92	val_92	92
-95	val_95	NULL
-95	val_95	NULL
-95	val_95	95
-95	val_95	95
-96	val_96	NULL
-96	val_96	96
-97	val_97	NULL
-97	val_97	NULL
-97	val_97	97
-97	val_97	97
-98	val_98	NULL
-98	val_98	NULL
-98	val_98	98
-98	val_98	98
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a
deleted file mode 100644
index dcf40c875b9adfbc8f817a65326c75fa438aa86f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a
+++ /dev/null
@@ -1,15 +0,0 @@
-tableName:partition_test_partitioned
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=101
-inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat
-columns:struct columns { string key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string dt}
-totalNumberFiles:1
-totalFileSize:284
-maxFileSize:284
-minFileSize:284
-lastAccessTime:0
-lastUpdateTime:1390900729000
-
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712
deleted file mode 100644
index 9b9389353dd5fe68457364ba12e61b55692bb207..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712
+++ /dev/null
@@ -1,25 +0,0 @@
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882
deleted file mode 100644
index 9b9389353dd5fe68457364ba12e61b55692bb207..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882
+++ /dev/null
@@ -1,25 +0,0 @@
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3
deleted file mode 100644
index 9639a1e84c61585694b483348c40257616ff2e87..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3
+++ /dev/null
@@ -1,50 +0,0 @@
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5
deleted file mode 100644
index 707036ebbc76cdacae63616113e3762a505f4f90..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5
+++ /dev/null
@@ -1,15 +0,0 @@
-tableName:partition_test_partitioned
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned
-inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-columns:struct columns { string key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string dt}
-totalNumberFiles:3
-totalFileSize:1415
-maxFileSize:895
-minFileSize:236
-lastAccessTime:0
-lastUpdateTime:1390900762000
-
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55
deleted file mode 100644
index 06316a924c38bde4a861bdbdd3f1151c53dd6605..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55
+++ /dev/null
@@ -1,15 +0,0 @@
-tableName:partition_test_partitioned
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=100
-inputformat:org.apache.hadoop.mapred.TextInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-columns:struct columns { string key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string dt}
-totalNumberFiles:1
-totalFileSize:236
-maxFileSize:236
-minFileSize:236
-lastAccessTime:0
-lastUpdateTime:1390900762000
-
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a
deleted file mode 100644
index 735d41e54e958ac73b23696ebb04e8c3940898bd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a
+++ /dev/null
@@ -1,15 +0,0 @@
-tableName:partition_test_partitioned
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=101
-inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat
-columns:struct columns { string key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string dt}
-totalNumberFiles:1
-totalFileSize:284
-maxFileSize:284
-minFileSize:284
-lastAccessTime:0
-lastUpdateTime:1390900762000
-
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63
deleted file mode 100644
index 27dfc622c9bb6d29b56efc926efc609fe1c9308b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63
+++ /dev/null
@@ -1,15 +0,0 @@
-tableName:partition_test_partitioned
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=102
-inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-columns:struct columns { string key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string dt}
-totalNumberFiles:1
-totalFileSize:895
-maxFileSize:895
-minFileSize:895
-lastAccessTime:0
-lastUpdateTime:1390900762000
-
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5
deleted file mode 100644
index 1812e0d53e443feae2f6b5ce82ac8798a71da89e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5
+++ /dev/null
@@ -1,15 +0,0 @@
-tableName:partition_test_partitioned
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned
-inputformat:org.apache.hadoop.mapred.TextInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-columns:struct columns { string key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string dt}
-totalNumberFiles:1
-totalFileSize:236
-maxFileSize:236
-minFileSize:236
-lastAccessTime:0
-lastUpdateTime:1390900706000
-
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712
deleted file mode 100644
index 9b9389353dd5fe68457364ba12e61b55692bb207..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712
+++ /dev/null
@@ -1,25 +0,0 @@
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882
deleted file mode 100644
index 9b9389353dd5fe68457364ba12e61b55692bb207..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882
+++ /dev/null
@@ -1,25 +0,0 @@
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a
deleted file mode 100644
index 9b9389353dd5fe68457364ba12e61b55692bb207..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a
+++ /dev/null
@@ -1,25 +0,0 @@
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3
deleted file mode 100644
index fca99e91bbd8f15cfc2d4c13233d7a6e78a691b9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3
+++ /dev/null
@@ -1,75 +0,0 @@
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f b/sql/hive/src/test/resources/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f
deleted file mode 100644
index fca99e91bbd8f15cfc2d4c13233d7a6e78a691b9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f
+++ /dev/null
@@ -1,75 +0,0 @@
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55
deleted file mode 100644
index 353225751161315bd33a59ee6359ae456164d82c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55
+++ /dev/null
@@ -1,15 +0,0 @@
-tableName:partition_test_partitioned
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=100
-inputformat:org.apache.hadoop.mapred.TextInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-columns:struct columns { string key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string dt}
-totalNumberFiles:1
-totalFileSize:236
-maxFileSize:236
-minFileSize:236
-lastAccessTime:0
-lastUpdateTime:1390900706000
-
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712
deleted file mode 100644
index 9b9389353dd5fe68457364ba12e61b55692bb207..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712
+++ /dev/null
@@ -1,25 +0,0 @@
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3
deleted file mode 100644
index 9b9389353dd5fe68457364ba12e61b55692bb207..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3
+++ /dev/null
@@ -1,25 +0,0 @@
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5
deleted file mode 100644
index a02458b88bf52e08f60c6a3fcc5ef9ca70e29491..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5
+++ /dev/null
@@ -1,15 +0,0 @@
-tableName:partition_test_partitioned
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned
-inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat
-columns:struct columns { string key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string dt}
-totalNumberFiles:2
-totalFileSize:520
-maxFileSize:284
-minFileSize:236
-lastAccessTime:0
-lastUpdateTime:1390900729000
-
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55
deleted file mode 100644
index 301bff44316ffbb9593f901e71951e5e962af253..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55
+++ /dev/null
@@ -1,15 +0,0 @@
-tableName:partition_test_partitioned
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=100
-inputformat:org.apache.hadoop.mapred.TextInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-columns:struct columns { string key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string dt}
-totalNumberFiles:1
-totalFileSize:236
-maxFileSize:236
-minFileSize:236
-lastAccessTime:0
-lastUpdateTime:1390900729000
-
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e
deleted file mode 100644
index 001841c8a1cd4f38c7d01efea039b2e1832e315e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e
+++ /dev/null
@@ -1,2 +0,0 @@
-238	val_238	1
-238	val_238	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf
deleted file mode 100644
index 63f35fd827de3eaa593882e42e7bf26bb0a927d9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf
+++ /dev/null
@@ -1,2 +0,0 @@
-476.0	val_238
-476.0	val_238
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf
deleted file mode 100644
index 63f35fd827de3eaa593882e42e7bf26bb0a927d9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf
+++ /dev/null
@@ -1,2 +0,0 @@
-476.0	val_238
-476.0	val_238
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e
deleted file mode 100644
index f75126345f351dbdb97302346c31b481f6169b2a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e
+++ /dev/null
@@ -1,2 +0,0 @@
-238	val_238	NULL	1
-238	val_238	NULL	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b
deleted file mode 100644
index de31196d97c3f11fd8d0b2b8a5451d3120d2315d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b
+++ /dev/null
@@ -1,2 +0,0 @@
-476	val_238
-476	val_238
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index 11542e84a3768bd8aff8cd093efc780340f98e5e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,2 +0,0 @@
-238	val_238	NULL	1
-238	val_238	NULL	1
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index 6d512a4f0bdc4f56e6a1ad8e34edf294b59edaca..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,2 +0,0 @@
-238	val_238	1
-238	val_238	1
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b
deleted file mode 100644
index 53aab40f88b50746be31e3eb3c9251de156375ff..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b
+++ /dev/null
@@ -1,2 +0,0 @@
-476.0	val_238
-476.0	val_238
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b
deleted file mode 100644
index de31196d97c3f11fd8d0b2b8a5451d3120d2315d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b
+++ /dev/null
@@ -1,2 +0,0 @@
-476	val_238
-476	val_238
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index 6d512a4f0bdc4f56e6a1ad8e34edf294b59edaca..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,2 +0,0 @@
-238	val_238	1
-238	val_238	1
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b
deleted file mode 100644
index 1b97612ce3f5aaaefd3d5ea36450b7ccf9b720bf..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b
+++ /dev/null
@@ -1,4 +0,0 @@
-476	val_238
-476	val_238
-194	val_97
-194	val_97
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index f8e13d5235028316e9fbb2b1f0e92d645cb25c1d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,4 +0,0 @@
-238	val_238	NULL	1
-238	val_238	NULL	1
-97	val_97	NULL	2
-97	val_97	NULL	2
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236
deleted file mode 100644
index 919a66a94c5cbbf007977404cb10d8199a03a2f8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236
+++ /dev/null
@@ -1,6 +0,0 @@
-476	val_238	NULL
-476	val_238	NULL
-194	val_97	NULL
-194	val_97	NULL
-400	val_200	val_200
-400	val_200	val_200
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index 45c169f0d8330172e49425be18fd986d5ec2115e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,6 +0,0 @@
-238	val_238	NULL	1
-238	val_238	NULL	1
-97	val_97	NULL	2
-97	val_97	NULL	2
-200	val_200	val_200	3
-200	val_200	val_200	3
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index 6d512a4f0bdc4f56e6a1ad8e34edf294b59edaca..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,2 +0,0 @@
-238	val_238	1
-238	val_238	1
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b
deleted file mode 100644
index 53aab40f88b50746be31e3eb3c9251de156375ff..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b
+++ /dev/null
@@ -1,2 +0,0 @@
-476.0	val_238
-476.0	val_238
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b
deleted file mode 100644
index de31196d97c3f11fd8d0b2b8a5451d3120d2315d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b
+++ /dev/null
@@ -1,2 +0,0 @@
-476	val_238
-476	val_238
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index 6d512a4f0bdc4f56e6a1ad8e34edf294b59edaca..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,2 +0,0 @@
-238	val_238	1
-238	val_238	1
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4
deleted file mode 100644
index b6a7d89c68e0ca66e96a9a51892cc33db66fb8a3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4
+++ /dev/null
@@ -1 +0,0 @@
-16
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d
deleted file mode 100644
index b6a7d89c68e0ca66e96a9a51892cc33db66fb8a3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d
+++ /dev/null
@@ -1 +0,0 @@
-16
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15
deleted file mode 100644
index 425151f3a411f5e088d7753e7c8d016303b1b9d1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15
+++ /dev/null
@@ -1 +0,0 @@
-40
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967
deleted file mode 100644
index 425151f3a411f5e088d7753e7c8d016303b1b9d1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967
+++ /dev/null
@@ -1 +0,0 @@
-40
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1
deleted file mode 100644
index 425151f3a411f5e088d7753e7c8d016303b1b9d1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1
+++ /dev/null
@@ -1 +0,0 @@
-40
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309
deleted file mode 100644
index c739b42c4d2ce23786c5350641d0adbf5fa7d6b2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309
+++ /dev/null
@@ -1 +0,0 @@
-44
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index 11542e84a3768bd8aff8cd093efc780340f98e5e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,2 +0,0 @@
-238	val_238	NULL	1
-238	val_238	NULL	1
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47
deleted file mode 100644
index 025abe4ec330b15612ebb593174a886479494681..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47
+++ /dev/null
@@ -1,3 +0,0 @@
-476	val_238	NULL	1
-476	val_238	NULL	1
-172	val_86	val_86	2
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index f067c1ed602dc348967f56d8d750b1f4583ff42e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,3 +0,0 @@
-238	val_238	NULL	1
-238	val_238	NULL	1
-86	val_86	val_86	2
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index 6d512a4f0bdc4f56e6a1ad8e34edf294b59edaca..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,2 +0,0 @@
-238	val_238	1
-238	val_238	1
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b
deleted file mode 100644
index 53aab40f88b50746be31e3eb3c9251de156375ff..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b
+++ /dev/null
@@ -1,2 +0,0 @@
-476.0	val_238
-476.0	val_238
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b
deleted file mode 100644
index de31196d97c3f11fd8d0b2b8a5451d3120d2315d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b
+++ /dev/null
@@ -1,2 +0,0 @@
-476	val_238
-476	val_238
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index 6d512a4f0bdc4f56e6a1ad8e34edf294b59edaca..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,2 +0,0 @@
-238	val_238	1
-238	val_238	1
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b
deleted file mode 100644
index de31196d97c3f11fd8d0b2b8a5451d3120d2315d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b
+++ /dev/null
@@ -1,2 +0,0 @@
-476	val_238
-476	val_238
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index 11542e84a3768bd8aff8cd093efc780340f98e5e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,2 +0,0 @@
-238	val_238	NULL	1
-238	val_238	NULL	1
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47
deleted file mode 100644
index 025abe4ec330b15612ebb593174a886479494681..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47
+++ /dev/null
@@ -1,3 +0,0 @@
-476	val_238	NULL	1
-476	val_238	NULL	1
-172	val_86	val_86	2
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index f067c1ed602dc348967f56d8d750b1f4583ff42e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,3 +0,0 @@
-238	val_238	NULL	1
-238	val_238	NULL	1
-86	val_86	val_86	2
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index 6d512a4f0bdc4f56e6a1ad8e34edf294b59edaca..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,2 +0,0 @@
-238	val_238	1
-238	val_238	1
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b
deleted file mode 100644
index 53aab40f88b50746be31e3eb3c9251de156375ff..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b
+++ /dev/null
@@ -1,2 +0,0 @@
-476.0	val_238
-476.0	val_238
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b
deleted file mode 100644
index de31196d97c3f11fd8d0b2b8a5451d3120d2315d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b
+++ /dev/null
@@ -1,2 +0,0 @@
-476	val_238
-476	val_238
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd
deleted file mode 100644
index 6d512a4f0bdc4f56e6a1ad8e34edf294b59edaca..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd
+++ /dev/null
@@ -1,2 +0,0 @@
-238	val_238	1
-238	val_238	1
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b
deleted file mode 100644
index de31196d97c3f11fd8d0b2b8a5451d3120d2315d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b
+++ /dev/null
@@ -1,2 +0,0 @@
-476	val_238
-476	val_238
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 b/sql/hive/src/test/resources/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 b/sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46
deleted file mode 100644
index d00491fd7e5bb6fa28c517a0bb32b8b506539d4d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46
+++ /dev/null
@@ -1 +0,0 @@
-1
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7
deleted file mode 100644
index 325e26b3d9737fda7dbdb1fcc38c338f659d6db1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7
+++ /dev/null
@@ -1,75 +0,0 @@
-238	val_238	100
-NULL		100
-311	val_311	100
-NULL	val_27	100
-NULL	val_165	100
-NULL	val_409	100
-255	val_255	100
-278	val_278	100
-98	val_98	100
-NULL	val_484	100
-NULL	val_265	100
-NULL	val_193	100
-401	val_401	100
-150	val_150	100
-273	val_273	100
-224		100
-369		100
-66	val_66	100
-128		100
-213	val_213	100
-146	val_146	100
-406	val_406	100
-NULL		100
-NULL		100
-NULL		100
-238	val_238	101
-NULL		101
-311	val_311	101
-NULL	val_27	101
-NULL	val_165	101
-NULL	val_409	101
-255	val_255	101
-278	val_278	101
-98	val_98	101
-NULL	val_484	101
-NULL	val_265	101
-NULL	val_193	101
-401	val_401	101
-150	val_150	101
-273	val_273	101
-224		101
-369		101
-66	val_66	101
-128		101
-213	val_213	101
-146	val_146	101
-406	val_406	101
-NULL		101
-NULL		101
-NULL		101
-238	val_238	102
-NULL		102
-311	val_311	102
-NULL	val_27	102
-NULL	val_165	102
-NULL	val_409	102
-255	val_255	102
-278	val_278	102
-98	val_98	102
-NULL	val_484	102
-NULL	val_265	102
-NULL	val_193	102
-401	val_401	102
-150	val_150	102
-273	val_273	102
-224		102
-369		102
-66	val_66	102
-128		102
-213	val_213	102
-146	val_146	102
-406	val_406	102
-NULL		102
-NULL		102
-NULL		102
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7
deleted file mode 100644
index 325e26b3d9737fda7dbdb1fcc38c338f659d6db1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7
+++ /dev/null
@@ -1,75 +0,0 @@
-238	val_238	100
-NULL		100
-311	val_311	100
-NULL	val_27	100
-NULL	val_165	100
-NULL	val_409	100
-255	val_255	100
-278	val_278	100
-98	val_98	100
-NULL	val_484	100
-NULL	val_265	100
-NULL	val_193	100
-401	val_401	100
-150	val_150	100
-273	val_273	100
-224		100
-369		100
-66	val_66	100
-128		100
-213	val_213	100
-146	val_146	100
-406	val_406	100
-NULL		100
-NULL		100
-NULL		100
-238	val_238	101
-NULL		101
-311	val_311	101
-NULL	val_27	101
-NULL	val_165	101
-NULL	val_409	101
-255	val_255	101
-278	val_278	101
-98	val_98	101
-NULL	val_484	101
-NULL	val_265	101
-NULL	val_193	101
-401	val_401	101
-150	val_150	101
-273	val_273	101
-224		101
-369		101
-66	val_66	101
-128		101
-213	val_213	101
-146	val_146	101
-406	val_406	101
-NULL		101
-NULL		101
-NULL		101
-238	val_238	102
-NULL		102
-311	val_311	102
-NULL	val_27	102
-NULL	val_165	102
-NULL	val_409	102
-255	val_255	102
-278	val_278	102
-98	val_98	102
-NULL	val_484	102
-NULL	val_265	102
-NULL	val_193	102
-401	val_401	102
-150	val_150	102
-273	val_273	102
-224		102
-369		102
-66	val_66	102
-128		102
-213	val_213	102
-146	val_146	102
-406	val_406	102
-NULL		102
-NULL		102
-NULL		102
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882
deleted file mode 100644
index 79c8f8e614a1fa2cd686178786f6c2c0bec30083..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882
+++ /dev/null
@@ -1,25 +0,0 @@
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a
deleted file mode 100644
index f487b340cd55b03e7a269359b2b1221eaac4ea0d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:partition_test_partitioned
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/partition_test_partitioned/dt=101
-inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat
-columns:struct columns { string key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string dt}
-totalNumberFiles:1
-totalFileSize:284
-maxFileSize:284
-minFileSize:284
-lastAccessTime:0
-lastUpdateTime:1388799388000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63
deleted file mode 100644
index 0c7739c2b9fbb62189d47447c2f70b74516c4da0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:partition_test_partitioned
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/partition_test_partitioned/dt=102
-inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-columns:struct columns { string key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string dt}
-totalNumberFiles:1
-totalFileSize:895
-maxFileSize:895
-minFileSize:895
-lastAccessTime:0
-lastUpdateTime:1388799405000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a
deleted file mode 100644
index 79c8f8e614a1fa2cd686178786f6c2c0bec30083..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a
+++ /dev/null
@@ -1,25 +0,0 @@
-238
-NULL
-311
-NULL
-NULL
-NULL
-255
-278
-98
-NULL
-NULL
-NULL
-401
-150
-273
-224
-369
-66
-128
-213
-146
-406
-NULL
-NULL
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a
deleted file mode 100644
index 0c6fbc997de7a7e1dd8eb62b0dff319ed5a63ab3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:partition_test_partitioned
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/partition_test_partitioned/dt=101
-inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-columns:struct columns { string key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string dt}
-totalNumberFiles:1
-totalFileSize:895
-maxFileSize:895
-minFileSize:895
-lastAccessTime:0
-lastUpdateTime:1388799434000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf
+++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745
index f0d140e18c0530503ff81295a27587b78b41d4a6..eb4c6a843cb5d87645a54fd9315a857bfdbd2915 100644
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745
+++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745
@@ -1,2 +1,2 @@
 101	25
-102	25
\ No newline at end of file
+102	25
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745
index 6425bae2a6a8f983ad45c7393924e19905a28b23..95846abf28b2f44a322ab2ea7beb234c73c5a136 100644
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745
+++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745
@@ -1,3 +1,3 @@
 101	25
 102	25
-103	25
\ No newline at end of file
+103	25
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf
+++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f
index d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 100644
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f
+++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f
@@ -1 +1 @@
-2
\ No newline at end of file
+2
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0
index d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 100644
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0
+++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0
@@ -1 +1 @@
-2
\ No newline at end of file
+2
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf
+++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2
index 3f10ffe7a4c473619c926cfb1e8d95e726e5a0ec..60d3b2f4a4cd5f1637eba020358bfe5ecb5edcf2 100644
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2
+++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2
@@ -1 +1 @@
-15
\ No newline at end of file
+15
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362
index d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f 100644
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362
+++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362
@@ -1 +1 @@
-2
\ No newline at end of file
+2
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b
deleted file mode 100644
index 5e06930239fea48c2a51edfe01cc11bc4a30e9ad..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b
+++ /dev/null
@@ -1,20 +0,0 @@
-0	val_0	1
-0	val_0	1
-0	val_0	1
-0	val_0	2
-0	val_0	2
-0	val_0	2
-0	val_0	3
-0	val_0	3
-0	val_0	3
-10	val_10	1
-10	val_10	2
-10	val_10	3
-100	val_100	1
-100	val_100	1
-100	val_100	2
-100	val_100	2
-100	val_100	3
-100	val_100	3
-103	val_103	1
-103	val_103	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd
deleted file mode 100644
index 1bd9063a9c63cb25648475d6e79a99012e0574d9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd
+++ /dev/null
@@ -1,20 +0,0 @@
-0.0	val_0	1
-0.0	val_0	1
-0.0	val_0	1
-0.0	val_0	2
-0.0	val_0	2
-0.0	val_0	2
-0.0	val_0	3
-0.0	val_0	3
-0.0	val_0	3
-4.0	val_2	1
-4.0	val_2	2
-4.0	val_2	3
-8.0	val_4	1
-8.0	val_4	2
-8.0	val_4	3
-10.0	val_5	1
-10.0	val_5	1
-10.0	val_5	1
-10.0	val_5	2
-10.0	val_5	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf
+++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b
index f259b4eefc6087c7d7792527bf001d0b23bd8b9a..967305a18236ec8a74acc105f4efb8afb1a2a941 100644
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b
+++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b
@@ -17,4 +17,4 @@
 104	val_104	1
 104	val_104	1
 104	val_104	2
-104	val_104	2
\ No newline at end of file
+104	val_104	2
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd
index 89afce150e7a94a6d9e40aa7ea4dee7d979c5489..f26a7375a6819e7c7a450caf1073c372bde00356 100644
--- a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd
+++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd
@@ -17,4 +17,4 @@
 16.0	val_8	1
 16.0	val_8	2
 18.0	val_9	1
-18.0	val_9	2
\ No newline at end of file
+18.0	val_9	2
diff --git a/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 b/sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5
index 55b2a1b47afa26d51a1d5af1d10afd419391c6dc..ef98fd7908a011ef645911e4ee3289e7c990b3dc 100644
--- a/sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5
+++ b/sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5
@@ -493,4 +493,4 @@
 403
 400
 200
-97
\ No newline at end of file
+97
diff --git a/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 b/sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5
index 55b2a1b47afa26d51a1d5af1d10afd419391c6dc..ef98fd7908a011ef645911e4ee3289e7c990b3dc 100644
--- a/sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5
+++ b/sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5
@@ -493,4 +493,4 @@
 403
 400
 200
-97
\ No newline at end of file
+97
diff --git a/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b b/sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b
index 878c86cb46b8a454b98f2dd3080ac0842d32a690..d2aff0f701be9d18641a80bbaebc3f6a77c6e26c 100644
--- a/sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b
+++ b/sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b
@@ -133,4 +133,4 @@
 480	3
 489	4
 492	2
-498	3
\ No newline at end of file
+498	3
diff --git a/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b b/sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b
index 878c86cb46b8a454b98f2dd3080ac0842d32a690..d2aff0f701be9d18641a80bbaebc3f6a77c6e26c 100644
--- a/sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b
+++ b/sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b
@@ -133,4 +133,4 @@
 480	3
 489	4
 492	2
-498	3
\ No newline at end of file
+498	3
diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c b/sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c
index c87107a2f11683e489e8ff577f09483ef1c74571..416fbfb9e5228d12016cf4657ba9e3e6a7e22381 100644
--- a/sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c
+++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c
@@ -22,4 +22,4 @@ NULL	NULL	NULL
 NULL	NULL	NULL
 NULL	NULL	NULL
 NULL	NULL	NULL
-NULL	NULL	NULL
\ No newline at end of file
+NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c b/sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c
index c87107a2f11683e489e8ff577f09483ef1c74571..416fbfb9e5228d12016cf4657ba9e3e6a7e22381 100644
--- a/sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c
+++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c
@@ -22,4 +22,4 @@ NULL	NULL	NULL
 NULL	NULL	NULL
 NULL	NULL	NULL
 NULL	NULL	NULL
-NULL	NULL	NULL
\ No newline at end of file
+NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 b/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5
index e37d32abba426c06b752a5e53f48c595c84e9270..83b33d238dab9943201aaf267f701e8ea5fc9268 100644
--- a/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5
+++ b/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5
@@ -1 +1 @@
-1000
\ No newline at end of file
+1000
diff --git a/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d b/sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d
index e1659e6072577a9ec77c0b900b9925b970e5bdff..f005a4fe5bf86fc7b87875093932cf47c935871d 100644
--- a/sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d
+++ b/sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d
@@ -126,4 +126,4 @@ val_395
 val_396
 val_397
 val_399
-val_4
\ No newline at end of file
+val_4
diff --git a/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d b/sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d
index e1659e6072577a9ec77c0b900b9925b970e5bdff..f005a4fe5bf86fc7b87875093932cf47c935871d 100644
--- a/sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d
+++ b/sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d
@@ -126,4 +126,4 @@ val_395
 val_396
 val_397
 val_399
-val_4
\ No newline at end of file
+val_4
diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f b/sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f
index 7e66866e2dd603bca041df4a1be7088689b81343..ab02a73437ed7a645ec3d758ce9e4b490f43caf6 100644
--- a/sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f
+++ b/sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f
@@ -2,4 +2,4 @@ val_4	1
 val_399	2
 val_396	3
 val_277	4
-val_348	5
\ No newline at end of file
+val_348	5
diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f b/sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f
index 7e66866e2dd603bca041df4a1be7088689b81343..ab02a73437ed7a645ec3d758ce9e4b490f43caf6 100644
--- a/sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f
+++ b/sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f
@@ -2,4 +2,4 @@ val_4	1
 val_399	2
 val_396	3
 val_277	4
-val_348	5
\ No newline at end of file
+val_348	5
diff --git a/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 b/sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998
index fbce9efa766d7dfc17f2c1242a6929c5b960bea2..a29747aef804612cacdfe7c973f7b8beefb41ea9 100644
--- a/sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998
+++ b/sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998
@@ -709,4 +709,4 @@
 200	val_200
 200	val_200
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 b/sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998
index fbce9efa766d7dfc17f2c1242a6929c5b960bea2..a29747aef804612cacdfe7c973f7b8beefb41ea9 100644
--- a/sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998
+++ b/sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998
@@ -709,4 +709,4 @@
 200	val_200
 200	val_200
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 b/sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9
index 9e6c4359e78a86197e087e093a2ec6246ecee202..f2748faa4ea7e1a1c10d4970705c485c7cb3dc22 100644
--- a/sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9
+++ b/sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9
@@ -1707,4 +1707,4 @@
 97	val_97
 97	val_97
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 b/sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9
index 9e6c4359e78a86197e087e093a2ec6246ecee202..f2748faa4ea7e1a1c10d4970705c485c7cb3dc22 100644
--- a/sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9
+++ b/sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9
@@ -1707,4 +1707,4 @@
 97	val_97
 97	val_97
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b b/sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b
index 0e11aea14d61de13bccd24fe7e6b529ebde8113d..91f2ce2b6178715b1deb9b4c6c107bffb57afa88 100644
--- a/sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b
+++ b/sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b
@@ -1764,4 +1764,4 @@
 97	val_97
 97	val_97
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b b/sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b
index 0e11aea14d61de13bccd24fe7e6b529ebde8113d..91f2ce2b6178715b1deb9b4c6c107bffb57afa88 100644
--- a/sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b
+++ b/sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b
@@ -1764,4 +1764,4 @@
 97	val_97
 97	val_97
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 b/sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524
index 997f37b76bedb35597ae630e73ae3d1c65ba9507..c0ffb7aeca9b6171e807c9c5d70e9fd222bfa833 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524
@@ -3,4 +3,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 b/sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524
index 997f37b76bedb35597ae630e73ae3d1c65ba9507..c0ffb7aeca9b6171e807c9c5d70e9fd222bfa833 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524
@@ -3,4 +3,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 b/sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36
index 997f37b76bedb35597ae630e73ae3d1c65ba9507..c0ffb7aeca9b6171e807c9c5d70e9fd222bfa833 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36
@@ -3,4 +3,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 b/sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36
index 997f37b76bedb35597ae630e73ae3d1c65ba9507..c0ffb7aeca9b6171e807c9c5d70e9fd222bfa833 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36
@@ -3,4 +3,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 b/sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0
index b3bf95dd32bac3168a76ae8526e26bafa6875226..9365b77dec0652e468ae189205de5bebb1f915e3 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0
@@ -3,4 +3,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-19	val_19	19	val_19
\ No newline at end of file
+19	val_19	19	val_19
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 b/sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0
index b3bf95dd32bac3168a76ae8526e26bafa6875226..9365b77dec0652e468ae189205de5bebb1f915e3 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0
@@ -3,4 +3,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-19	val_19	19	val_19
\ No newline at end of file
+19	val_19	19	val_19
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe b/sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe
index d4a363c49aeafa32a24dced76e202903ba670561..daa4ea309e2c1a1e8f9daf1d372ffd82c28a0773 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe
@@ -7,4 +7,4 @@
 18	val_18	18	val_18	18
 18	val_18	18	val_18	18
 18	val_18	18	val_18	18
-18	val_18	18	val_18	18
\ No newline at end of file
+18	val_18	18	val_18	18
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe b/sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe
index d4a363c49aeafa32a24dced76e202903ba670561..daa4ea309e2c1a1e8f9daf1d372ffd82c28a0773 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe
@@ -7,4 +7,4 @@
 18	val_18	18	val_18	18
 18	val_18	18	val_18	18
 18	val_18	18	val_18	18
-18	val_18	18	val_18	18
\ No newline at end of file
+18	val_18	18	val_18	18
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 b/sql/hive/src/test/resources/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 b/sql/hive/src/test/resources/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745
deleted file mode 100644
index 0190981db84ed467a736e28ab815ae43b1258667..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745
+++ /dev/null
@@ -1,84 +0,0 @@
-0	val_0
-0	val_0
-0	val_0
-10	val_10
-11	val_11
-12	val_12
-12	val_12
-15	val_15
-15	val_15
-17	val_17
-18	val_18
-18	val_18
-19	val_19
-2	val_2
-20	val_20
-24	val_24
-24	val_24
-26	val_26
-26	val_26
-27	val_27
-28	val_28
-30	val_30
-33	val_33
-34	val_34
-35	val_35
-35	val_35
-35	val_35
-37	val_37
-37	val_37
-4	val_4
-41	val_41
-42	val_42
-42	val_42
-43	val_43
-44	val_44
-47	val_47
-5	val_5
-5	val_5
-5	val_5
-51	val_51
-51	val_51
-53	val_53
-54	val_54
-57	val_57
-58	val_58
-58	val_58
-64	val_64
-65	val_65
-66	val_66
-67	val_67
-67	val_67
-69	val_69
-70	val_70
-70	val_70
-70	val_70
-72	val_72
-72	val_72
-74	val_74
-76	val_76
-76	val_76
-77	val_77
-78	val_78
-8	val_8
-80	val_80
-82	val_82
-83	val_83
-83	val_83
-84	val_84
-84	val_84
-85	val_85
-86	val_86
-87	val_87
-9	val_9
-90	val_90
-90	val_90
-90	val_90
-92	val_92
-95	val_95
-95	val_95
-96	val_96
-97	val_97
-97	val_97
-98	val_98
-98	val_98
diff --git a/sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 b/sql/hive/src/test/resources/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 b/sql/hive/src/test/resources/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745
deleted file mode 100644
index 0190981db84ed467a736e28ab815ae43b1258667..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745
+++ /dev/null
@@ -1,84 +0,0 @@
-0	val_0
-0	val_0
-0	val_0
-10	val_10
-11	val_11
-12	val_12
-12	val_12
-15	val_15
-15	val_15
-17	val_17
-18	val_18
-18	val_18
-19	val_19
-2	val_2
-20	val_20
-24	val_24
-24	val_24
-26	val_26
-26	val_26
-27	val_27
-28	val_28
-30	val_30
-33	val_33
-34	val_34
-35	val_35
-35	val_35
-35	val_35
-37	val_37
-37	val_37
-4	val_4
-41	val_41
-42	val_42
-42	val_42
-43	val_43
-44	val_44
-47	val_47
-5	val_5
-5	val_5
-5	val_5
-51	val_51
-51	val_51
-53	val_53
-54	val_54
-57	val_57
-58	val_58
-58	val_58
-64	val_64
-65	val_65
-66	val_66
-67	val_67
-67	val_67
-69	val_69
-70	val_70
-70	val_70
-70	val_70
-72	val_72
-72	val_72
-74	val_74
-76	val_76
-76	val_76
-77	val_77
-78	val_78
-8	val_8
-80	val_80
-82	val_82
-83	val_83
-83	val_83
-84	val_84
-84	val_84
-85	val_85
-86	val_86
-87	val_87
-9	val_9
-90	val_90
-90	val_90
-90	val_90
-92	val_92
-95	val_95
-95	val_95
-96	val_96
-97	val_97
-97	val_97
-98	val_98
-98	val_98
diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9
+++ b/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e b/sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e
index 4c3919232c73eb5ae01b43d2e342c7d82ff5b59a..c30d4a581ba6c302dfc9a6c14519cd586603849e 100644
--- a/sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e
+++ b/sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e
@@ -168,4 +168,4 @@
 90	val_90
 403	val_403
 400	val_400
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803
+++ b/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e b/sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e
index 4c3919232c73eb5ae01b43d2e342c7d82ff5b59a..c30d4a581ba6c302dfc9a6c14519cd586603849e 100644
--- a/sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e
+++ b/sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e
@@ -168,4 +168,4 @@
 90	val_90
 403	val_403
 400	val_400
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 b/sql/hive/src/test/resources/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 b/sql/hive/src/test/resources/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f b/sql/hive/src/test/resources/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c b/sql/hive/src/test/resources/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 b/sql/hive/src/test/resources/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 b/sql/hive/src/test/resources/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 b/sql/hive/src/test/resources/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 b/sql/hive/src/test/resources/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 b/sql/hive/src/test/resources/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c b/sql/hive/src/test/resources/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 b/sql/hive/src/test/resources/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c b/sql/hive/src/test/resources/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 b/sql/hive/src/test/resources/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 b/sql/hive/src/test/resources/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229
deleted file mode 100644
index d183a30ddf3b6c8a9a42077b59683a4ec43b4db4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229
+++ /dev/null
@@ -1,36 +0,0 @@
-238	val_238	2008-04-08	11
-86	val_86	2008-04-08	11
-311	val_311	2008-04-08	11
-27	val_27	2008-04-08	11
-165	val_165	2008-04-08	11
-409	val_409	2008-04-08	11
-255	val_255	2008-04-08	11
-278	val_278	2008-04-08	11
-98	val_98	2008-04-08	11
-238	val_238	2008-04-08	12
-86	val_86	2008-04-08	12
-311	val_311	2008-04-08	12
-27	val_27	2008-04-08	12
-165	val_165	2008-04-08	12
-409	val_409	2008-04-08	12
-255	val_255	2008-04-08	12
-278	val_278	2008-04-08	12
-98	val_98	2008-04-08	12
-238	val_238	2008-04-09	11
-86	val_86	2008-04-09	11
-311	val_311	2008-04-09	11
-27	val_27	2008-04-09	11
-165	val_165	2008-04-09	11
-409	val_409	2008-04-09	11
-255	val_255	2008-04-09	11
-278	val_278	2008-04-09	11
-98	val_98	2008-04-09	11
-238	val_238	2008-04-09	12
-86	val_86	2008-04-09	12
-311	val_311	2008-04-09	12
-27	val_27	2008-04-09	12
-165	val_165	2008-04-09	12
-409	val_409	2008-04-09	12
-255	val_255	2008-04-09	12
-278	val_278	2008-04-09	12
-98	val_98	2008-04-09	12
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 b/sql/hive/src/test/resources/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 b/sql/hive/src/test/resources/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7
deleted file mode 100644
index 5b2461e35b5f4efed3a7d892590b7533ccaf1729..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7
+++ /dev/null
@@ -1,36 +0,0 @@
-238	val_238	2008-04-08	11	0
-238	val_238	2008-04-08	11	0
-86	val_86	2008-04-08	11	12
-311	val_311	2008-04-08	11	22
-311	val_311	2008-04-08	11	22
-311	val_311	2008-04-08	11	22
-27	val_27	2008-04-08	11	34
-165	val_165	2008-04-08	11	44
-165	val_165	2008-04-08	11	44
-238	val_238	2008-04-08	12	0
-238	val_238	2008-04-08	12	0
-86	val_86	2008-04-08	12	12
-311	val_311	2008-04-08	12	22
-311	val_311	2008-04-08	12	22
-311	val_311	2008-04-08	12	22
-27	val_27	2008-04-08	12	34
-165	val_165	2008-04-08	12	44
-165	val_165	2008-04-08	12	44
-238	val_238	2008-04-09	11	0
-238	val_238	2008-04-09	11	0
-86	val_86	2008-04-09	11	12
-311	val_311	2008-04-09	11	22
-311	val_311	2008-04-09	11	22
-311	val_311	2008-04-09	11	22
-27	val_27	2008-04-09	11	34
-165	val_165	2008-04-09	11	44
-165	val_165	2008-04-09	11	44
-238	val_238	2008-04-09	12	0
-238	val_238	2008-04-09	12	0
-86	val_86	2008-04-09	12	12
-311	val_311	2008-04-09	12	22
-311	val_311	2008-04-09	12	22
-311	val_311	2008-04-09	12	22
-27	val_27	2008-04-09	12	34
-165	val_165	2008-04-09	12	44
-165	val_165	2008-04-09	12	44
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/ppr_pushdown-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/ppr_pushdown-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b b/sql/hive/src/test/resources/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/ppr_pushdown-1-855b235f9c760ba9d6d0200bfd1ad08b
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394
rename to sql/hive/src/test/resources/golden/ppr_pushdown-1-855b235f9c760ba9d6d0200bfd1ad08b
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 b/sql/hive/src/test/resources/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 b/sql/hive/src/test/resources/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/ppr_pushdown-10-2957fd9b211cee5f0372525a1de55c19
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1
rename to sql/hive/src/test/resources/golden/ppr_pushdown-10-2957fd9b211cee5f0372525a1de55c19
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 b/sql/hive/src/test/resources/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/ppr_pushdown-11-b8d6f0ffc8294497c792b26958adee45
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/ppr_pushdown-11-b8d6f0ffc8294497c792b26958adee45
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 b/sql/hive/src/test/resources/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/ppr_pushdown-12-ab1b9a0cdd9586c96d8856a9d632b89c
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc
rename to sql/hive/src/test/resources/golden/ppr_pushdown-12-ab1b9a0cdd9586c96d8856a9d632b89c
diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 b/sql/hive/src/test/resources/golden/ppr_pushdown-13-2c316c67fd494d878fbbea107d283c3b
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993
rename to sql/hive/src/test/resources/golden/ppr_pushdown-13-2c316c67fd494d878fbbea107d283c3b
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 b/sql/hive/src/test/resources/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/ppr_pushdown-14-53b4be82a1538844d03b200429efa02b
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/ppr_pushdown-14-53b4be82a1538844d03b200429efa02b
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c b/sql/hive/src/test/resources/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/ppr_pushdown-15-71ab93d38ed2908069091c7c8cc0aba1
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc
rename to sql/hive/src/test/resources/golden/ppr_pushdown-15-71ab93d38ed2908069091c7c8cc0aba1
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb b/sql/hive/src/test/resources/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae b/sql/hive/src/test/resources/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/ppr_pushdown-16-855cb54d28034fdb20a3615ee0918d63
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a
rename to sql/hive/src/test/resources/golden/ppr_pushdown-16-855cb54d28034fdb20a3615ee0918d63
diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/ppr_pushdown-17-d3ccf2722a8b7281fcee61b2544772c8
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394
rename to sql/hive/src/test/resources/golden/ppr_pushdown-17-d3ccf2722a8b7281fcee61b2544772c8
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 b/sql/hive/src/test/resources/golden/ppr_pushdown-18-fcd10fbfc55a2c0aa843fe618f9613c6
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6
rename to sql/hive/src/test/resources/golden/ppr_pushdown-18-fcd10fbfc55a2c0aa843fe618f9613c6
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 b/sql/hive/src/test/resources/golden/ppr_pushdown-19-ff36e3978571ac05f11e8322c024e4b6
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6
rename to sql/hive/src/test/resources/golden/ppr_pushdown-19-ff36e3978571ac05f11e8322c024e4b6
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 b/sql/hive/src/test/resources/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/ppr_pushdown-2-f689aaf15b08c433d1e93ce977a9b6d5
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1
rename to sql/hive/src/test/resources/golden/ppr_pushdown-2-f689aaf15b08c433d1e93ce977a9b6d5
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a b/sql/hive/src/test/resources/golden/ppr_pushdown-20-481005cf034ef3d7b998da32eb82aa9a
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a
rename to sql/hive/src/test/resources/golden/ppr_pushdown-20-481005cf034ef3d7b998da32eb82aa9a
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 b/sql/hive/src/test/resources/golden/ppr_pushdown-21-9073825e8b9804331f780980d1f9fa92
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92
rename to sql/hive/src/test/resources/golden/ppr_pushdown-21-9073825e8b9804331f780980d1f9fa92
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d b/sql/hive/src/test/resources/golden/ppr_pushdown-22-9cb28f0d3a434c9d1ab039192906ec9d
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d
rename to sql/hive/src/test/resources/golden/ppr_pushdown-22-9cb28f0d3a434c9d1ab039192906ec9d
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d b/sql/hive/src/test/resources/golden/ppr_pushdown-23-678f50025924fee7b59f66e2abdb472d
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d
rename to sql/hive/src/test/resources/golden/ppr_pushdown-23-678f50025924fee7b59f66e2abdb472d
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 b/sql/hive/src/test/resources/golden/ppr_pushdown-24-4d5bfa800ba434f464b07bf9b5d39f20
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20
rename to sql/hive/src/test/resources/golden/ppr_pushdown-24-4d5bfa800ba434f464b07bf9b5d39f20
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 b/sql/hive/src/test/resources/golden/ppr_pushdown-25-22663f09ea1c1bc303756067e84df5a7
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7
rename to sql/hive/src/test/resources/golden/ppr_pushdown-25-22663f09ea1c1bc303756067e84df5a7
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b b/sql/hive/src/test/resources/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/ppr_pushdown-26-e789d9b469aa1fffe4ce0a15a8c1fb9b
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/ppr_pushdown-26-e789d9b469aa1fffe4ce0a15a8c1fb9b
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 b/sql/hive/src/test/resources/golden/ppr_pushdown-27-8065c18e387fd8bffae19a80af8dc1d4
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4
rename to sql/hive/src/test/resources/golden/ppr_pushdown-27-8065c18e387fd8bffae19a80af8dc1d4
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 b/sql/hive/src/test/resources/golden/ppr_pushdown-28-b72de558c88ae91460989938000e0d27
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27
rename to sql/hive/src/test/resources/golden/ppr_pushdown-28-b72de558c88ae91460989938000e0d27
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 b/sql/hive/src/test/resources/golden/ppr_pushdown-29-7217735d357770af4ffee730e4e9add4
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4
rename to sql/hive/src/test/resources/golden/ppr_pushdown-29-7217735d357770af4ffee730e4e9add4
diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/ppr_pushdown-3-5eede4874e678021938683fc2f5dc900
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc
rename to sql/hive/src/test/resources/golden/ppr_pushdown-3-5eede4874e678021938683fc2f5dc900
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 b/sql/hive/src/test/resources/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea b/sql/hive/src/test/resources/golden/ppr_pushdown-30-630e2f7918b7727fc4ca057fa21e2eea
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea
rename to sql/hive/src/test/resources/golden/ppr_pushdown-30-630e2f7918b7727fc4ca057fa21e2eea
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 b/sql/hive/src/test/resources/golden/ppr_pushdown-31-2e062414293b643ea4d7f6df92f939e4
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4
rename to sql/hive/src/test/resources/golden/ppr_pushdown-31-2e062414293b643ea4d7f6df92f939e4
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf b/sql/hive/src/test/resources/golden/ppr_pushdown-32-5eba4bf10315099129eae319d73636cf
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf
rename to sql/hive/src/test/resources/golden/ppr_pushdown-32-5eba4bf10315099129eae319d73636cf
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b b/sql/hive/src/test/resources/golden/ppr_pushdown-33-35af0585a4f98bc222c786688cb6de6b
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b
rename to sql/hive/src/test/resources/golden/ppr_pushdown-33-35af0585a4f98bc222c786688cb6de6b
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 b/sql/hive/src/test/resources/golden/ppr_pushdown-34-a5594625510703427ab8bae5d0563f73
similarity index 100%
rename from sql/hive/src/test/resources/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73
rename to sql/hive/src/test/resources/golden/ppr_pushdown-34-a5594625510703427ab8bae5d0563f73
diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/ppr_pushdown-4-c5c542f8ee81cd0afd44e67fc7b4d306
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a
rename to sql/hive/src/test/resources/golden/ppr_pushdown-4-c5c542f8ee81cd0afd44e67fc7b4d306
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 b/sql/hive/src/test/resources/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de b/sql/hive/src/test/resources/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/ppr_pushdown-5-f54bebec398f0fdfdbc0393123dba234
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394
rename to sql/hive/src/test/resources/golden/ppr_pushdown-5-f54bebec398f0fdfdbc0393123dba234
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d b/sql/hive/src/test/resources/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/ppr_pushdown-6-78af12432bcbf99d4a0d41c25f964de
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1
rename to sql/hive/src/test/resources/golden/ppr_pushdown-6-78af12432bcbf99d4a0d41c25f964de
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 b/sql/hive/src/test/resources/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 b/sql/hive/src/test/resources/golden/ppr_pushdown-7-58724fbe96a0b3deceef20c8cc5e318d
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873
rename to sql/hive/src/test/resources/golden/ppr_pushdown-7-58724fbe96a0b3deceef20c8cc5e318d
diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/ppr_pushdown-8-14570f946e75924d7926c809485951d1
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd
rename to sql/hive/src/test/resources/golden/ppr_pushdown-8-14570f946e75924d7926c809485951d1
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c b/sql/hive/src/test/resources/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 b/sql/hive/src/test/resources/golden/ppr_pushdown-9-3ba325662296fc455f07f1c835495e4c
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2
rename to sql/hive/src/test/resources/golden/ppr_pushdown-9-3ba325662296fc455f07f1c835495e4c
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 b/sql/hive/src/test/resources/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/ppr_pushdown2-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/ppr_pushdown2-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b b/sql/hive/src/test/resources/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 b/sql/hive/src/test/resources/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 b/sql/hive/src/test/resources/golden/ppr_pushdown2-1-855b235f9c760ba9d6d0200bfd1ad08b
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873
rename to sql/hive/src/test/resources/golden/ppr_pushdown2-1-855b235f9c760ba9d6d0200bfd1ad08b
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-10-4fa4ba4c739b6f44975d41e4705d4389 b/sql/hive/src/test/resources/golden/ppr_pushdown2-10-4fa4ba4c739b6f44975d41e4705d4389
new file mode 100644
index 0000000000000000000000000000000000000000..f50a5fea8dd5d3a4c5043e73812de0a7298029b6
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-10-4fa4ba4c739b6f44975d41e4705d4389
@@ -0,0 +1 @@
+1	1	2
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c b/sql/hive/src/test/resources/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b b/sql/hive/src/test/resources/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/ppr_pushdown2-11-ab3e46183487096745d7d90e3020e94c
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd
rename to sql/hive/src/test/resources/golden/ppr_pushdown2-11-ab3e46183487096745d7d90e3020e94c
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a b/sql/hive/src/test/resources/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/ppr_pushdown2-12-b10b9e14f5a7a7a92c0c68df6dbc656a
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8
rename to sql/hive/src/test/resources/golden/ppr_pushdown2-12-b10b9e14f5a7a7a92c0c68df6dbc656a
diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/ppr_pushdown2-13-154f9859bd0822e287fbfdff12fd45ff
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc
rename to sql/hive/src/test/resources/golden/ppr_pushdown2-13-154f9859bd0822e287fbfdff12fd45ff
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 b/sql/hive/src/test/resources/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 b/sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8
deleted file mode 100644
index e3e04ee48543db61f7da48870ee824e1248a9dad..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8
+++ /dev/null
@@ -1 +0,0 @@
-3	1	2	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 b/sql/hive/src/test/resources/golden/ppr_pushdown2-14-cf4a30b5c8329d8d79ddf762f318fbb3
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873
rename to sql/hive/src/test/resources/golden/ppr_pushdown2-14-cf4a30b5c8329d8d79ddf762f318fbb3
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 b/sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41
deleted file mode 100644
index c458b0f57aba946067eea11c61e4c180099bbda7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41
+++ /dev/null
@@ -1 +0,0 @@
-2	1	1	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-15-8d01597374157f2d3d066840983ba1f8 b/sql/hive/src/test/resources/golden/ppr_pushdown2-15-8d01597374157f2d3d066840983ba1f8
new file mode 100644
index 0000000000000000000000000000000000000000..3def25c3c65afa341d255f30a972d237cf3d389f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-15-8d01597374157f2d3d066840983ba1f8
@@ -0,0 +1 @@
+3	1	2	1
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb b/sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb
deleted file mode 100644
index c458b0f57aba946067eea11c61e4c180099bbda7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb
+++ /dev/null
@@ -1 +0,0 @@
-2	1	1	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-16-5614065e1b8e709f68be4fa67666f41 b/sql/hive/src/test/resources/golden/ppr_pushdown2-16-5614065e1b8e709f68be4fa67666f41
new file mode 100644
index 0000000000000000000000000000000000000000..55c794b56ec9aab3f9e7cc8b0e494e260edb6914
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-16-5614065e1b8e709f68be4fa67666f41
@@ -0,0 +1 @@
+2	1	1	2
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-17-3a67618e47c977f58c9dd8f4b9a576eb b/sql/hive/src/test/resources/golden/ppr_pushdown2-17-3a67618e47c977f58c9dd8f4b9a576eb
new file mode 100644
index 0000000000000000000000000000000000000000..55c794b56ec9aab3f9e7cc8b0e494e260edb6914
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-17-3a67618e47c977f58c9dd8f4b9a576eb
@@ -0,0 +1 @@
+2	1	1	2
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe b/sql/hive/src/test/resources/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe
deleted file mode 100644
index 63511415ddf555fa422f64dd418fd28c0ab34b48..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe
+++ /dev/null
@@ -1,2 +0,0 @@
-3	1	2	1
-1	1	2	3
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-18-44e5f3ac566f60d8b17ef19c18a11ebe b/sql/hive/src/test/resources/golden/ppr_pushdown2-18-44e5f3ac566f60d8b17ef19c18a11ebe
new file mode 100644
index 0000000000000000000000000000000000000000..8d13286371dab0cc306100beb2ad2015cb725a7a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-18-44e5f3ac566f60d8b17ef19c18a11ebe
@@ -0,0 +1,2 @@
+3	1	2	1
+1	1	2	3
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 b/sql/hive/src/test/resources/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775
deleted file mode 100644
index c592b5d505b0ec5ccb70a066d2c55500ef13419b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775
+++ /dev/null
@@ -1,3 +0,0 @@
-2	1	1	2
-3	1	2	1
-1	1	2	3
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 b/sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5
deleted file mode 100644
index e3e04ee48543db61f7da48870ee824e1248a9dad..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5
+++ /dev/null
@@ -1 +0,0 @@
-3	1	2	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-19-e2c7e9b01ec95dfcc685827e24d66775 b/sql/hive/src/test/resources/golden/ppr_pushdown2-19-e2c7e9b01ec95dfcc685827e24d66775
new file mode 100644
index 0000000000000000000000000000000000000000..2bc7fedb12a5079b5351e8e81cf7a37910e42bc7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-19-e2c7e9b01ec95dfcc685827e24d66775
@@ -0,0 +1,3 @@
+2	1	1	2
+3	1	2	1
+1	1	2	3
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e b/sql/hive/src/test/resources/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/ppr_pushdown2-2-ffa167b63d612a4986d02f5c0623ea7b
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd
rename to sql/hive/src/test/resources/golden/ppr_pushdown2-2-ffa167b63d612a4986d02f5c0623ea7b
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-20-bd7e1917f8d2cf50c062a22ef3fa15b5 b/sql/hive/src/test/resources/golden/ppr_pushdown2-20-bd7e1917f8d2cf50c062a22ef3fa15b5
new file mode 100644
index 0000000000000000000000000000000000000000..3def25c3c65afa341d255f30a972d237cf3d389f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-20-bd7e1917f8d2cf50c062a22ef3fa15b5
@@ -0,0 +1 @@
+3	1	2	1
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 b/sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229
deleted file mode 100644
index e3e04ee48543db61f7da48870ee824e1248a9dad..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229
+++ /dev/null
@@ -1 +0,0 @@
-3	1	2	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-21-ece6fe0efc1e658b36ddc10f0653d229 b/sql/hive/src/test/resources/golden/ppr_pushdown2-21-ece6fe0efc1e658b36ddc10f0653d229
new file mode 100644
index 0000000000000000000000000000000000000000..3def25c3c65afa341d255f30a972d237cf3d389f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-21-ece6fe0efc1e658b36ddc10f0653d229
@@ -0,0 +1 @@
+3	1	2	1
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c b/sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c
deleted file mode 100644
index 679926f7d738c65c34b5febecc66656390f34c4c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c
+++ /dev/null
@@ -1 +0,0 @@
-2	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 b/sql/hive/src/test/resources/golden/ppr_pushdown2-3-c7753746c190414723d66a8f876499c7
similarity index 100%
rename from sql/hive/src/test/resources/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8
rename to sql/hive/src/test/resources/golden/ppr_pushdown2-3-c7753746c190414723d66a8f876499c7
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-4-1886675984606b6c972c4a26dca6fd2c b/sql/hive/src/test/resources/golden/ppr_pushdown2-4-1886675984606b6c972c4a26dca6fd2c
new file mode 100644
index 0000000000000000000000000000000000000000..bfde072a7963c7ce7e808b99a04ccaf84e02de59
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-4-1886675984606b6c972c4a26dca6fd2c
@@ -0,0 +1 @@
+2	2
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 b/sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910
deleted file mode 100644
index fcc7be2cb12aa71e6cd58011ed3b101567f737f5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910
+++ /dev/null
@@ -1 +0,0 @@
-22	22
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-5-33b137b28e7246ec3c2acb937c638910 b/sql/hive/src/test/resources/golden/ppr_pushdown2-5-33b137b28e7246ec3c2acb937c638910
new file mode 100644
index 0000000000000000000000000000000000000000..38212d1943095875165d71265be16c889beabf5f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-5-33b137b28e7246ec3c2acb937c638910
@@ -0,0 +1 @@
+22	22
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 b/sql/hive/src/test/resources/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 b/sql/hive/src/test/resources/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba b/sql/hive/src/test/resources/golden/ppr_pushdown2-6-b5a2518af801f95fe52a75dfc1d3e867
similarity index 100%
rename from sql/hive/src/test/resources/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba
rename to sql/hive/src/test/resources/golden/ppr_pushdown2-6-b5a2518af801f95fe52a75dfc1d3e867
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 b/sql/hive/src/test/resources/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 b/sql/hive/src/test/resources/golden/ppr_pushdown2-7-e89a8d1f66fdf9ce68f345de1f728c5b
similarity index 100%
rename from sql/hive/src/test/resources/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3
rename to sql/hive/src/test/resources/golden/ppr_pushdown2-7-e89a8d1f66fdf9ce68f345de1f728c5b
diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 b/sql/hive/src/test/resources/golden/ppr_pushdown2-8-4507a3f200b3ce384191c91acd324dc7
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5
rename to sql/hive/src/test/resources/golden/ppr_pushdown2-8-4507a3f200b3ce384191c91acd324dc7
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 b/sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40
deleted file mode 100644
index 3ebc6d7fa23754dc74d4a9de6ea9966af43efa8a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40
+++ /dev/null
@@ -1 +0,0 @@
-2	2	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 b/sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389
deleted file mode 100644
index a4544ab84afa375f46ef27646996189bc3d15b0d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389
+++ /dev/null
@@ -1 +0,0 @@
-1	1	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-9-c86940e70f705f50e9091c257ee2bb40 b/sql/hive/src/test/resources/golden/ppr_pushdown2-9-c86940e70f705f50e9091c257ee2bb40
new file mode 100644
index 0000000000000000000000000000000000000000..699fa0cd95c4fbd031ae87e3cb8e2cc6aa87349c
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-9-c86940e70f705f50e9091c257ee2bb40
@@ -0,0 +1 @@
+2	2	1
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a b/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 b/sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0
index e22f840876d2e0f96bdd5994934d4ea84580f008..546fd0b0e105121ff1ce4c1a68ac222015c24463 100644
--- a/sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0
@@ -37,4 +37,4 @@
 5	val_5	2008-04-09	12
 2	val_2	2008-04-09	12
 5	val_5	2008-04-09	12
-9	val_9	2008-04-09	12
\ No newline at end of file
+9	val_9	2008-04-09	12
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb b/sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb
index 355ed1617e20018010132230b44d49a02a186571..2857cdf0aba86636e320800d63c5ebf3f5975b08 100644
--- a/sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb
@@ -1997,4 +1997,4 @@
 403	val_403	2008-04-09	12
 400	val_400	2008-04-09	12
 200	val_200	2008-04-09	12
-97	val_97	2008-04-09	12
\ No newline at end of file
+97	val_97	2008-04-09	12
diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f b/sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f
index 57005044dde388c7e59f1046e657c3b60e7f4f9a..7f3ca6e01ea062c1ce84acb55a23f0c247068c37 100644
--- a/sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f
+++ b/sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f
@@ -1997,4 +1997,4 @@
 403
 400
 200
-97
\ No newline at end of file
+97
diff --git a/sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 b/sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/print_header-1-8540676fc16ac91f3629c40f393a890a b/sql/hive/src/test/resources/golden/print_header-1-8540676fc16ac91f3629c40f393a890a
deleted file mode 100644
index 943e1be13b61544f7c2d5214fbd5e277f5d41b89..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/print_header-1-8540676fc16ac91f3629c40f393a890a
+++ /dev/null
@@ -1,10 +0,0 @@
-0	val_0	0	val_0
-0	val_0	0	val_0
-0	val_0	0	val_0
-0	val_0	0	val_0
-0	val_0	0	val_0
-0	val_0	0	val_0
-0	val_0	0	val_0
-0	val_0	0	val_0
-0	val_0	0	val_0
-0	val_0	2	val_2
diff --git a/sql/hive/src/test/resources/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 b/sql/hive/src/test/resources/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93
deleted file mode 100644
index 951e74db0fe23bf2c6fe1439f3d400574904526f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93
+++ /dev/null
@@ -1,10 +0,0 @@
-0	0.0
-2	2.0
-4	4.0
-5	15.0
-8	8.0
-9	9.0
-10	10.0
-11	11.0
-12	24.0
-15	30.0
diff --git a/sql/hive/src/test/resources/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 b/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04
+++ b/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f b/sql/hive/src/test/resources/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 b/sql/hive/src/test/resources/golden/progress_1-2-b6c8c8fc9df98af4dead5efabf5f162c
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70
rename to sql/hive/src/test/resources/golden/progress_1-2-b6c8c8fc9df98af4dead5efabf5f162c
diff --git a/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 b/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4
index 0b3e0a69a8c7977c396c644a388038e7ebe3fc3e..e9c02dad1826afb00c3759de476520d6de144bb3 100644
--- a/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4
+++ b/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4
@@ -1 +1 @@
-5000
\ No newline at end of file
+5000
diff --git a/sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 b/sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18
index eb63636d0bfd47435afccaca4502fda3183a84b7..41b92dbf0214a39315476f57897238e241b1612a 100644
--- a/sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18
+++ b/sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18
@@ -1,9 +1,9 @@
-col                 	string              	None                
-p                   	string              	None                
+col                 	string              	                    
+p                   	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-p                   	string              	None                
+p                   	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[p1], dbName:default, tableName:tbl2, createTime:1388801768, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388801769, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1388801769})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[p1], dbName:default, tableName:tbl2, createTime:1413890256, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413890256, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1413890256, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1})	
diff --git a/sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d
index 964ede006ad21940f82cda66832a54eb87cc0f51..27254dd97006de99f9a79d27bcaa79b13955ebf6 100644
--- a/sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d
+++ b/sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d
@@ -1,9 +1,9 @@
-col                 	string              	None                
-p                   	string              	None                
+col                 	string              	                    
+p                   	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-p                   	string              	None                
+p                   	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1388801768, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388801770, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1388801770}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1413890256, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413890256, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1413890256}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d
index 68b03670f5f80c8cf666f42109cfe2b9d4f629ef..4d09f8c357ea78652b23b0df8d97c1c2ed121f3f 100644
--- a/sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d
+++ b/sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d
@@ -1,9 +1,9 @@
-col                 	string              	None                
-p                   	string              	None                
+col                 	string              	                    
+p                   	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-p                   	string              	None                
+p                   	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1388801768, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388801770, PROTECT_MODE=OFFLINE,NO_DROP, transient_lastDdlTime=1388801770}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1413890256, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413890256, PROTECT_MODE=OFFLINE,NO_DROP, transient_lastDdlTime=1413890256}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d
index ec18ce74115352171af1275b04eb780d2c46d2a7..dbc128d2a1d8cf423f3706c0b34b912325763616 100644
--- a/sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d
+++ b/sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d
@@ -1,9 +1,9 @@
-col                 	string              	None                
-p                   	string              	None                
+col                 	string              	                    
+p                   	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-p                   	string              	None                
+p                   	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1388801768, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388801770, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1388801770}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1413890256, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413890256, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1413890256}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d
index a6202ac5fc0eb57afad459281dccfa620be79d0f..7774c774cc8a1efa9cb2cf467539be94b5df998e 100644
--- a/sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d
+++ b/sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d
@@ -1,9 +1,9 @@
-col                 	string              	None                
-p                   	string              	None                
+col                 	string              	                    
+p                   	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-p                   	string              	None                
+p                   	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1388801768, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388801771, transient_lastDdlTime=1388801771}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1413890256, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413890256, transient_lastDdlTime=1413890256}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 b/sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18
index 0a6cebbbd0b825e8651ae256b282fc4ceed53f05..567b9b3a5d22883bd97506952557f5808da79c8c 100644
--- a/sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18
+++ b/sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18
@@ -1,9 +1,9 @@
-col                 	string              	None                
-p                   	string              	None                
+col                 	string              	                    
+p                   	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-p                   	string              	None                
+p                   	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[p1], dbName:default, tableName:tbl2, createTime:1388801768, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388801783, transient_lastDdlTime=1388801783})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[p1], dbName:default, tableName:tbl2, createTime:1413890256, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413890263, transient_lastDdlTime=1413890263, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1})	
diff --git a/sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 b/sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510
index a1ff1f8341f302c00fab354398f00c65c16f3a1d..be56722166fe405ffc8fab31aec106d65fa35d9c 100644
--- a/sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510
+++ b/sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510
@@ -1,3 +1,3 @@
-col                 	string              	None                
+col                 	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:tbl1, dbName:default, owner:marmbrus, createTime:1388801746, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388801756, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1388801756}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:tbl1, dbName:default, owner:marmbrus, createTime:1413890242, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413890249, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1413890249, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 b/sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510
index 9098c9419556b2a5deb3f96e19876b43562989ff..a3c3c67860fdf8a7d5d39a6ae45460a91cf4be9c 100644
--- a/sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510
+++ b/sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510
@@ -1,3 +1,3 @@
-col                 	string              	None                
+col                 	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:tbl1, dbName:default, owner:marmbrus, createTime:1388801746, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388801756, transient_lastDdlTime=1388801756}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:tbl1, dbName:default, owner:marmbrus, createTime:1413890242, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413890249, transient_lastDdlTime=1413890249, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a b/sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a
index 3a2f20c6378839436e345cda0b474d35908e8031..e3c6c66098c10926c1bd9802c11b7cc893775ee7 100644
--- a/sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a
+++ b/sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a
@@ -20,4 +20,4 @@
 17	val_17	2000-04-09
 18	val_18	2000-04-09
 18	val_18	2000-04-09
-19	val_19	2000-04-09
\ No newline at end of file
+19	val_19	2000-04-09
diff --git a/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 b/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436
index 2f3fe0189d2c66d846a523526d9352e5a355ef23..10db2ad303244ecf39e4b5a7fba23267a3a62bf7 100644
--- a/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436
+++ b/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436
@@ -1 +1 @@
-aa;
\ No newline at end of file
+aa;
diff --git a/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb b/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb
index b5b5773c405b48235f24b489e56c5bd6522a4773..e0b3f1b09bd1819ed1f7ce2e75fc7400809f5350 100644
--- a/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb
+++ b/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb
@@ -1 +1 @@
-bb
\ No newline at end of file
+bb
diff --git a/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 b/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767
index 2652f5f42c003f125212dd61f95a3a8a37cb45d5..46c1d6125b7b4a120a61881204c2cb38fb442401 100644
--- a/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767
+++ b/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767
@@ -1 +1 @@
-cc
\ No newline at end of file
+cc
diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/quote2-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/quote2-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 b/sql/hive/src/test/resources/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 b/sql/hive/src/test/resources/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9
deleted file mode 100644
index ed1f53a6588d0f3e42996e292939552154097fa2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9
+++ /dev/null
@@ -1 +0,0 @@
-abc	abc	abc'	abc"	abc\	abc\	abc\'	abc\"	abc\\	abc\\	abc\\'	abc\\"	abc\\\	abc\\\	abc""""\	abc''''\	awk '{print NR"\t"$0}'	tab	tab	tab	tab
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 b/sql/hive/src/test/resources/golden/quote2-1-ea1a1d0c5f9a3248afbb65e6632c5118
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5
rename to sql/hive/src/test/resources/golden/quote2-1-ea1a1d0c5f9a3248afbb65e6632c5118
diff --git a/sql/hive/src/test/resources/golden/quote2-2-34f3c423b2fb1f0b11457f45a60042b9 b/sql/hive/src/test/resources/golden/quote2-2-34f3c423b2fb1f0b11457f45a60042b9
new file mode 100644
index 0000000000000000000000000000000000000000..4c8564d085999f10559c053c3042b83703310949
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/quote2-2-34f3c423b2fb1f0b11457f45a60042b9
@@ -0,0 +1 @@
+abc	abc	abc'	abc"	abc\	abc\	abc\'	abc\"	abc\\	abc\\	abc\\'	abc\\"	abc\\\	abc\\\	abc""""\	abc''''\	awk '{print NR"\t"$0}'	tab	tab	tab	tab
diff --git a/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e b/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e	
+++ b/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e	
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d b/sql/hive/src/test/resources/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 b/sql/hive/src/test/resources/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321
deleted file mode 100644
index 73b8edab368339270d51a394e1dad81298baa791..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321
+++ /dev/null
@@ -1,56 +0,0 @@
-409	val_409
-429	val_429
-209	val_209
-153	val_153
-203	val_203
-170	val_170
-489	val_489
-378	val_378
-221	val_221
-498	val_498
-469	val_469
-176	val_176
-176	val_176
-384	val_384
-217	val_217
-431	val_431
-51	val_51
-288	val_288
-457	val_457
-197	val_197
-77	val_77
-138	val_138
-277	val_277
-224	val_224
-309	val_309
-389	val_389
-331	val_331
-317	val_317
-336	val_336
-42	val_42
-458	val_458
-78	val_78
-453	val_453
-74	val_74
-103	val_103
-467	val_467
-202	val_202
-469	val_469
-44	val_44
-454	val_454
-70	val_70
-491	val_491
-199	val_199
-169	val_169
-310	val_310
-233	val_233
-133	val_133
-26	val_26
-134	val_134
-18	val_18
-298	val_298
-348	val_348
-469	val_469
-37	val_37
-152	val_152
-400	val_400
diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928
deleted file mode 100644
index e8738e9c837eec1bd6e84010ddd352f5350229a2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928
+++ /dev/null
@@ -1,116 +0,0 @@
-103	val_103	2008-04-08	11
-118	val_118	2008-04-08	12
-119	val_119	2008-04-08	12
-119	val_119	2008-04-08	12
-126	val_126	2008-04-08	12
-131	val_131	2008-04-08	12
-133	val_133	2008-04-08	11
-134	val_134	2008-04-08	11
-138	val_138	2008-04-08	11
-143	val_143	2008-04-08	12
-152	val_152	2008-04-08	11
-153	val_153	2008-04-08	11
-162	val_162	2008-04-08	12
-169	val_169	2008-04-08	11
-170	val_170	2008-04-08	11
-175	val_175	2008-04-08	12
-176	val_176	2008-04-08	11
-176	val_176	2008-04-08	11
-18	val_18	2008-04-08	11
-18	val_18	2008-04-08	12
-191	val_191	2008-04-08	12
-197	val_197	2008-04-08	11
-199	val_199	2008-04-08	11
-200	val_200	2008-04-08	12
-201	val_201	2008-04-08	12
-202	val_202	2008-04-08	11
-203	val_203	2008-04-08	11
-209	val_209	2008-04-08	11
-214	val_214	2008-04-08	12
-217	val_217	2008-04-08	11
-218	val_218	2008-04-08	12
-221	val_221	2008-04-08	11
-223	val_223	2008-04-08	12
-224	val_224	2008-04-08	11
-229	val_229	2008-04-08	12
-230	val_230	2008-04-08	12
-233	val_233	2008-04-08	11
-233	val_233	2008-04-08	12
-237	val_237	2008-04-08	12
-238	val_238	2008-04-08	12
-256	val_256	2008-04-08	12
-26	val_26	2008-04-08	11
-265	val_265	2008-04-08	12
-273	val_273	2008-04-08	12
-277	val_277	2008-04-08	11
-277	val_277	2008-04-08	12
-280	val_280	2008-04-08	12
-286	val_286	2008-04-08	12
-288	val_288	2008-04-08	11
-298	val_298	2008-04-08	11
-309	val_309	2008-04-08	11
-309	val_309	2008-04-08	12
-310	val_310	2008-04-08	11
-317	val_317	2008-04-08	11
-322	val_322	2008-04-08	12
-323	val_323	2008-04-08	12
-325	val_325	2008-04-08	12
-331	val_331	2008-04-08	11
-332	val_332	2008-04-08	12
-336	val_336	2008-04-08	11
-336	val_336	2008-04-08	12
-339	val_339	2008-04-08	12
-341	val_341	2008-04-08	12
-342	val_342	2008-04-08	12
-348	val_348	2008-04-08	11
-348	val_348	2008-04-08	12
-35	val_35	2008-04-08	12
-364	val_364	2008-04-08	12
-37	val_37	2008-04-08	11
-378	val_378	2008-04-08	11
-384	val_384	2008-04-08	11
-389	val_389	2008-04-08	11
-400	val_400	2008-04-08	11
-403	val_403	2008-04-08	12
-407	val_407	2008-04-08	12
-409	val_409	2008-04-08	11
-417	val_417	2008-04-08	12
-42	val_42	2008-04-08	11
-424	val_424	2008-04-08	12
-429	val_429	2008-04-08	11
-429	val_429	2008-04-08	12
-430	val_430	2008-04-08	12
-431	val_431	2008-04-08	11
-432	val_432	2008-04-08	12
-44	val_44	2008-04-08	11
-453	val_453	2008-04-08	11
-454	val_454	2008-04-08	11
-457	val_457	2008-04-08	11
-457	val_457	2008-04-08	12
-458	val_458	2008-04-08	11
-466	val_466	2008-04-08	12
-467	val_467	2008-04-08	11
-469	val_469	2008-04-08	11
-469	val_469	2008-04-08	11
-469	val_469	2008-04-08	11
-47	val_47	2008-04-08	12
-470	val_470	2008-04-08	12
-489	val_489	2008-04-08	11
-491	val_491	2008-04-08	11
-496	val_496	2008-04-08	12
-498	val_498	2008-04-08	11
-498	val_498	2008-04-08	12
-51	val_51	2008-04-08	11
-58	val_58	2008-04-08	12
-70	val_70	2008-04-08	11
-72	val_72	2008-04-08	12
-74	val_74	2008-04-08	11
-77	val_77	2008-04-08	11
-77	val_77	2008-04-08	12
-78	val_78	2008-04-08	11
-82	val_82	2008-04-08	12
-87	val_87	2008-04-08	12
-90	val_90	2008-04-08	12
-97	val_97	2008-04-08	12
-97	val_97	2008-04-08	12
-98	val_98	2008-04-08	12
diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e b/sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 b/sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 b/sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055
deleted file mode 100644
index d00491fd7e5bb6fa28c517a0bb32b8b506539d4d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055
+++ /dev/null
@@ -1 +0,0 @@
-1
diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a b/sql/hive/src/test/resources/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b b/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b
index d1b3011bffd911a28fce7e8618199762f9d85194..2d2bcebee4a8da3d9fac1bcf3c672bb2babb6dfa 100644
--- a/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b
+++ b/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b
@@ -1,2 +1,2 @@
-key                 	string              	None                
-value               	string              	None                
+key                 	string              	                    
+value               	string              	                    
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 b/sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 b/sql/hive/src/test/resources/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c b/sql/hive/src/test/resources/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 b/sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40
deleted file mode 100644
index b141fed81f2067fb09c7a1a2328865336f38a703..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40
+++ /dev/null
@@ -1 +0,0 @@
-53278638794
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 b/sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5
deleted file mode 100644
index b141fed81f2067fb09c7a1a2328865336f38a703..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5
+++ /dev/null
@@ -1 +0,0 @@
-53278638794
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 b/sql/hive/src/test/resources/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e b/sql/hive/src/test/resources/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee b/sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 b/sql/hive/src/test/resources/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e b/sql/hive/src/test/resources/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 b/sql/hive/src/test/resources/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 b/sql/hive/src/test/resources/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f b/sql/hive/src/test/resources/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 b/sql/hive/src/test/resources/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 b/sql/hive/src/test/resources/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb b/sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 b/sql/hive/src/test/resources/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 b/sql/hive/src/test/resources/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a b/sql/hive/src/test/resources/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 b/sql/hive/src/test/resources/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 b/sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685
deleted file mode 100644
index 8d25d618795b97a1f41e1d30b7245296e6d7185e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685
+++ /dev/null
@@ -1 +0,0 @@
--4208881187
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 b/sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592
deleted file mode 100644
index 8d25d618795b97a1f41e1d30b7245296e6d7185e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592
+++ /dev/null
@@ -1 +0,0 @@
--4208881187
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 b/sql/hive/src/test/resources/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 b/sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69
deleted file mode 100644
index 11ff946b46f0fc111b309b44e140c55439078840..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69
+++ /dev/null
@@ -1 +0,0 @@
-14412220296
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f b/sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f
deleted file mode 100644
index 11ff946b46f0fc111b309b44e140c55439078840..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f
+++ /dev/null
@@ -1 +0,0 @@
-14412220296
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb b/sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 b/sql/hive/src/test/resources/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b b/sql/hive/src/test/resources/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 b/sql/hive/src/test/resources/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 b/sql/hive/src/test/resources/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 b/sql/hive/src/test/resources/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 b/sql/hive/src/test/resources/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 b/sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69
deleted file mode 100644
index 11ff946b46f0fc111b309b44e140c55439078840..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69
+++ /dev/null
@@ -1 +0,0 @@
-14412220296
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f b/sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f
deleted file mode 100644
index 11ff946b46f0fc111b309b44e140c55439078840..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f
+++ /dev/null
@@ -1 +0,0 @@
-14412220296
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb b/sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 b/sql/hive/src/test/resources/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 b/sql/hive/src/test/resources/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 b/sql/hive/src/test/resources/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 b/sql/hive/src/test/resources/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 b/sql/hive/src/test/resources/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee b/sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee
index b0135533064c906f8d123e27b7b67af124c9c926..2918f17e964c06f60869420836ce580773c3dfc8 100644
--- a/sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee
+++ b/sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee
@@ -22,4 +22,4 @@ NULL	val_193
 406	val_406
 NULL	
 NULL	
-NULL	
\ No newline at end of file
+NULL	
diff --git a/sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 b/sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2
index 06b2b4d7e6d4767f51ce7559c1a3b99cd420b5b8..b52cff5c472e4cce9ece568cdb1b7f57616f6e3b 100644
--- a/sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2
+++ b/sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2
@@ -6,4 +6,4 @@ NULL	NULL	24	val_24
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b
+++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8
+++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d
+++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a
+++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590
index 55d948599907211562da45bf68242aa0462d556f..b70e127e82d056a30f56d6141cafa419ffef0184 100644
--- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590
+++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590
@@ -497,4 +497,4 @@
 497	val_497
 498	val_498
 498	val_498
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b
deleted file mode 100644
index 60878ffb77064ace9427b51d45234d9b94b04408..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b	
+++ /dev/null
@@ -1 +0,0 @@
-238	val_238
diff --git a/sql/hive/src/test/resources/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 b/sql/hive/src/test/resources/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767
+++ b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 b/sql/hive/src/test/resources/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db b/sql/hive/src/test/resources/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/regex_col-2-21564f64cdfd46098e1254380490701 b/sql/hive/src/test/resources/golden/regex_col-2-21564f64cdfd46098e1254380490701
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 b/sql/hive/src/test/resources/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 b/sql/hive/src/test/resources/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 b/sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa b/sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa
deleted file mode 100644
index d00491fd7e5bb6fa28c517a0bb32b8b506539d4d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa
+++ /dev/null
@@ -1 +0,0 @@
-1
diff --git a/sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505 b/sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/remote_script-3-4eb54a664e549614d56ca088c8867d b/sql/hive/src/test/resources/golden/remote_script-3-4eb54a664e549614d56ca088c8867d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b
index 726e0947d830223006675a38ca8ce0e19cb78543..017e14d2ebed42aa873a7bf3826c48414e6f3218 100644
--- a/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
-a                   	int                 	None                
-b                   	int                 	None                
-c                   	int                 	None                
\ No newline at end of file
+a                   	int                 	                    
+b                   	int                 	                    
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b
index 500201be8d922bf3f91f4c1f3fa925686090d2f2..a92663b0674bf029e326c1fb5bfb763b75699ee5 100644
--- a/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
-b                   	int                 	None                
+b                   	int                 	                    
 a1                  	int                 	test comment1       
-c                   	int                 	None                
\ No newline at end of file
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b
index 71af16a5fa4d79d1181a91745c8dc6c80678503d..899341a881857cfa2e942d64ca1076064f7734e7 100644
--- a/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
 a2                  	int                 	test comment2       
-b                   	int                 	None                
-c                   	int                 	None                
\ No newline at end of file
+b                   	int                 	                    
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b
index f5b2e72aeccab4540db8137bfaa14e070c45ed84..26b38dcc6d855f53ef829f100a7323e8646d1e3d 100644
--- a/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
-b                   	int                 	None                
+b                   	int                 	                    
 a                   	int                 	test comment2       
-c                   	int                 	None                
\ No newline at end of file
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c
index ecafeaea5f61ab99d238c32150ce31e27145f46c..85c1918f46567d9bb13c443b5142296d0d2fbc14 100644
--- a/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c
+++ b/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c
@@ -1,2 +1,2 @@
 src
-srcpart
\ No newline at end of file
+srcpart
diff --git a/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b
index 726e0947d830223006675a38ca8ce0e19cb78543..017e14d2ebed42aa873a7bf3826c48414e6f3218 100644
--- a/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
-a                   	int                 	None                
-b                   	int                 	None                
-c                   	int                 	None                
\ No newline at end of file
+a                   	int                 	                    
+b                   	int                 	                    
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b
index 17127eaec97550dd618b51cb22b0e5ca37cf88d6..2fbb615dd5994aff11654a9460aa3d7b4a70b2c5 100644
--- a/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
-a                   	string              	None                
-b                   	int                 	None                
-c                   	int                 	None                
\ No newline at end of file
+a                   	string              	                    
+b                   	int                 	                    
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b
index bfcefe4d180467b0e96074a7d9adde88adecbaf0..173fbad7b1eb3390314e96919bb7e1b983f797d4 100644
--- a/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
-a1                  	int                 	None                
-b                   	int                 	None                
-c                   	int                 	None                
\ No newline at end of file
+a1                  	int                 	                    
+b                   	int                 	                    
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b
index c436c39a16b8a4cfdfaceea598e9f2cfeeaa8354..bad9feb96a88687031c1db5f0cfd4718903fbdf3 100644
--- a/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
-a2                  	int                 	None                
-b                   	int                 	None                
-c                   	int                 	None                
\ No newline at end of file
+a2                  	int                 	                    
+b                   	int                 	                    
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b
index bb1507e7488f531af19a88697200f655578bde74..4f23db53afff27e6c87bb79782cbaaae4d72c1bb 100644
--- a/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
-b                   	int                 	None                
-a                   	int                 	None                
-c                   	int                 	None                
\ No newline at end of file
+b                   	int                 	                    
+a                   	int                 	                    
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b
index 17127eaec97550dd618b51cb22b0e5ca37cf88d6..2fbb615dd5994aff11654a9460aa3d7b4a70b2c5 100644
--- a/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
-a                   	string              	None                
-b                   	int                 	None                
-c                   	int                 	None                
\ No newline at end of file
+a                   	string              	                    
+b                   	int                 	                    
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b
index 500201be8d922bf3f91f4c1f3fa925686090d2f2..a92663b0674bf029e326c1fb5bfb763b75699ee5 100644
--- a/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
-b                   	int                 	None                
+b                   	int                 	                    
 a1                  	int                 	test comment1       
-c                   	int                 	None                
\ No newline at end of file
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b
index 71af16a5fa4d79d1181a91745c8dc6c80678503d..899341a881857cfa2e942d64ca1076064f7734e7 100644
--- a/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
 a2                  	int                 	test comment2       
-b                   	int                 	None                
-c                   	int                 	None                
\ No newline at end of file
+b                   	int                 	                    
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b
index f5b2e72aeccab4540db8137bfaa14e070c45ed84..26b38dcc6d855f53ef829f100a7323e8646d1e3d 100644
--- a/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
-b                   	int                 	None                
+b                   	int                 	                    
 a                   	int                 	test comment2       
-c                   	int                 	None                
\ No newline at end of file
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b
index bfcefe4d180467b0e96074a7d9adde88adecbaf0..173fbad7b1eb3390314e96919bb7e1b983f797d4 100644
--- a/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
-a1                  	int                 	None                
-b                   	int                 	None                
-c                   	int                 	None                
\ No newline at end of file
+a1                  	int                 	                    
+b                   	int                 	                    
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b
index c436c39a16b8a4cfdfaceea598e9f2cfeeaa8354..bad9feb96a88687031c1db5f0cfd4718903fbdf3 100644
--- a/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
-a2                  	int                 	None                
-b                   	int                 	None                
-c                   	int                 	None                
\ No newline at end of file
+a2                  	int                 	                    
+b                   	int                 	                    
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b
index bb1507e7488f531af19a88697200f655578bde74..4f23db53afff27e6c87bb79782cbaaae4d72c1bb 100644
--- a/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b
+++ b/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b
@@ -1,3 +1,3 @@
-b                   	int                 	None                
-a                   	int                 	None                
-c                   	int                 	None                
\ No newline at end of file
+b                   	int                 	                    
+a                   	int                 	                    
+c                   	int                 	                    
diff --git a/sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 b/sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 b/sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47
index 3a57720041fb345fec33000d5b7cbf8005c697db..d091388cd5e19263f1797a7e6cff6e73e25ea2b0 100644
--- a/sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47
+++ b/sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47
@@ -9,4 +9,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 b/sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6
index 19492fd335bcb6752ca9d2dfe37e21d1abf8362d..a86e9c5af723e131f574dc4c9b7901c3ac5641f7 100644
--- a/sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6
+++ b/sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6
@@ -9,4 +9,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f b/sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f
index 3a57720041fb345fec33000d5b7cbf8005c697db..d091388cd5e19263f1797a7e6cff6e73e25ea2b0 100644
--- a/sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f
+++ b/sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f
@@ -9,4 +9,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e b/sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e
index 19492fd335bcb6752ca9d2dfe37e21d1abf8362d..a86e9c5af723e131f574dc4c9b7901c3ac5641f7 100644
--- a/sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e
+++ b/sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e
@@ -9,4 +9,4 @@
 18	val_18	18	val_18
 18	val_18	18	val_18
 18	val_18	18	val_18
-18	val_18	18	val_18
\ No newline at end of file
+18	val_18	18	val_18
diff --git a/sql/hive/src/test/resources/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 b/sql/hive/src/test/resources/golden/sample2-1-a1d8184eab25b242a961533cc016efd1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-10-60eadbb52f8857830a3034952c631ace b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-10-60eadbb52f8857830a3034952c631ace
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-12-60018cae9a0476dc6a0ab4264310edb5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-12-60018cae9a0476dc6a0ab4264310edb5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-8-f9dd797f1c90e2108cfee585f443c132 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-8-f9dd797f1c90e2108cfee585f443c132
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-9-22fdd8380f2652de2492b34a425d46d7 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-9-22fdd8380f2652de2492b34a425d46d7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-10-22fdd8380f2652de2492b34a425d46d7 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-10-22fdd8380f2652de2492b34a425d46d7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-11-60eadbb52f8857830a3034952c631ace b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-11-60eadbb52f8857830a3034952c631ace
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-13-60018cae9a0476dc6a0ab4264310edb5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-13-60018cae9a0476dc6a0ab4264310edb5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-8-7a45282169e5a15d70ae0afb9e67ec9a b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-8-7a45282169e5a15d70ae0afb9e67ec9a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-9-f9dd797f1c90e2108cfee585f443c132 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-9-f9dd797f1c90e2108cfee585f443c132
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 b/sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64
deleted file mode 100644
index 0bc999a3e6aa2c0a9be8d7750b47540bab82fc84..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64
+++ /dev/null
@@ -1,2 +0,0 @@
-1
-1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef b/sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 b/sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0
deleted file mode 100644
index 0bc999a3e6aa2c0a9be8d7750b47540bab82fc84..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0
+++ /dev/null
@@ -1,2 +0,0 @@
-1
-1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 b/sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 b/sql/hive/src/test/resources/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 b/sql/hive/src/test/resources/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 b/sql/hive/src/test/resources/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f b/sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f
deleted file mode 100644
index 9cf5170e82d7d43dc8dd988f06215a0355cb0bd0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f
+++ /dev/null
@@ -1 +0,0 @@
-238	val_238	238	val_238
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 b/sql/hive/src/test/resources/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 b/sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5
deleted file mode 100644
index d00491fd7e5bb6fa28c517a0bb32b8b506539d4d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5
+++ /dev/null
@@ -1 +0,0 @@
-1
diff --git a/sql/hive/src/test/resources/golden/select from thrift based table-0-304c4992f5517febd10f43c57df4da49 b/sql/hive/src/test/resources/golden/select from thrift based table-0-304c4992f5517febd10f43c57df4da49
new file mode 100644
index 0000000000000000000000000000000000000000..a81afd7b04319dd631c845595b15df09c5d31978
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/select from thrift based table-0-304c4992f5517febd10f43c57df4da49	
@@ -0,0 +1,11 @@
+1712634731	record_0	[0,0,0]	["0","0","0"]	[{"myint":0,"mystring":"0","underscore_int":0}]	{"key_0":"value_0"}
+465985200	record_1	[1,2,3]	["10","100","1000"]	[{"myint":1,"mystring":"1","underscore_int":1}]	{"key_1":"value_1"}
+-751827638	record_2	[2,4,6]	["20","200","2000"]	[{"myint":4,"mystring":"8","underscore_int":2}]	{"key_2":"value_2"}
+477111222	record_3	[3,6,9]	["30","300","3000"]	[{"myint":9,"mystring":"27","underscore_int":3}]	{"key_3":"value_3"}
+-734328909	record_4	[4,8,12]	["40","400","4000"]	[{"myint":16,"mystring":"64","underscore_int":4}]	{"key_4":"value_4"}
+-1952710710	record_5	[5,10,15]	["50","500","5000"]	[{"myint":25,"mystring":"125","underscore_int":5}]	{"key_5":"value_5"}
+1244525190	record_6	[6,12,18]	["60","600","6000"]	[{"myint":36,"mystring":"216","underscore_int":6}]	{"key_6":"value_6"}
+-1461153973	record_7	[7,14,21]	["70","700","7000"]	[{"myint":49,"mystring":"343","underscore_int":7}]	{"key_7":"value_7"}
+1638581578	record_8	[8,16,24]	["80","800","8000"]	[{"myint":64,"mystring":"512","underscore_int":8}]	{"key_8":"value_8"}
+336964413	record_9	[9,18,27]	["90","900","9000"]	[{"myint":81,"mystring":"729","underscore_int":9}]	{"key_9":"value_9"}
+0	NULL	NULL	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 b/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145
index 5f4de85940513b9da7cfa5af9aa2eee989db923d..016f64cc26f2aa60723e8998a7a8ee8da582ea16 100644
--- a/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145
+++ b/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145
@@ -1 +1 @@
-0	val_0
\ No newline at end of file
+0	val_0
diff --git a/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e b/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e
index f35d9602e1a7b7da27644a48e1997c111e3bfd8c..f9aaa4d565f4a169e1033d0aa0aa7eb4a22e351f 100644
--- a/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e
+++ b/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e
@@ -1 +1 @@
-498
\ No newline at end of file
+498
diff --git a/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 b/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232
+++ b/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e b/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e
index f35d9602e1a7b7da27644a48e1997c111e3bfd8c..f9aaa4d565f4a169e1033d0aa0aa7eb4a22e351f 100644
--- a/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e
+++ b/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e
@@ -1 +1 @@
-498
\ No newline at end of file
+498
diff --git a/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde b/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde
+++ b/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e b/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e
index f35d9602e1a7b7da27644a48e1997c111e3bfd8c..f9aaa4d565f4a169e1033d0aa0aa7eb4a22e351f 100644
--- a/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e
+++ b/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e
@@ -1 +1 @@
-498
\ No newline at end of file
+498
diff --git a/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f b/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f
+++ b/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e b/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e
index d00ee7786a57c63f2b1eb9c5add6b102bc6af0ed..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391 100644
--- a/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e
+++ b/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e
@@ -1,22 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_CREATETABLE (TOK_TABNAME serde_regex) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL host TOK_STRING) (TOK_TABCOL identity TOK_STRING) (TOK_TABCOL user TOK_STRING) (TOK_TABCOL time TOK_STRING) (TOK_TABCOL request TOK_STRING) (TOK_TABCOL status TOK_STRING) (TOK_TABCOL size TOK_INT) (TOK_TABCOL referer TOK_STRING) (TOK_TABCOL agent TOK_STRING)) (TOK_TABLESERIALIZER (TOK_SERDENAME 'org.apache.hadoop.hive.serde2.RegexSerDe' (TOK_TABLEPROPERTIES (TOK_TABLEPROPLIST (TOK_TABLEPROPERTY "input.regex" "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) ([^ \"]*|\"[^\"]*\") (-|[0-9]*) (-|[0-9]*)(?: ([^ \"]*|\"[^\"]*\") ([^ \"]*|\"[^\"]*\"))?"))))) TOK_TBLTEXTFILE)
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Create Table Operator:
-        Create Table
-          columns: host string, identity string, user string, time string, request string, status string, size int, referer string, agent string
-          if not exists: false
-          input format: org.apache.hadoop.mapred.TextInputFormat
-          # buckets: -1
-          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
-          serde name: org.apache.hadoop.hive.serde2.RegexSerDe
-          serde properties:
-            input.regex ([^ ]*) ([^ ]*) ([^ ]*) (-|\[[^\]]*\]) ([^ "]*|"[^"]*") (-|[0-9]*) (-|[0-9]*)(?: ([^ "]*|"[^"]*") ([^ "]*|"[^"]*"))?
-          name: serde_regex
-          isExternal: false
-
-
diff --git a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429
index 3e290231c27e2bece9e08112cb7fcb94bddcf9f6..93cdc5c85645caa5f8c443d6276547cc5569f93c 100644
--- a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429
+++ b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429
@@ -1,5 +1,4 @@
 NULL	0
-NULL	0
 -1234567890.123456789	-1234567890
 -4400	4400
 -1255.49	-1255
@@ -11,6 +10,7 @@ NULL	0
 -0.3	0
 0	0
 0	0
+0	0
 0.01	0
 0.02	0
 0.1	0
@@ -18,7 +18,7 @@ NULL	0
 0.3	0
 0.33	0
 0.333	0
-0.9999999999999999999999999	1
+1	1
 1	1
 1	1
 1.12	1
diff --git a/sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9 b/sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 b/sql/hive/src/test/resources/golden/serde_regex-2-e84d30fcc6cf11e82b54ea63e7d1d611
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695
rename to sql/hive/src/test/resources/golden/serde_regex-2-e84d30fcc6cf11e82b54ea63e7d1d611
diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 b/sql/hive/src/test/resources/golden/serde_regex-3-3ee9e78ff563d6b48741a41885f92c81
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48
rename to sql/hive/src/test/resources/golden/serde_regex-3-3ee9e78ff563d6b48741a41885f92c81
diff --git a/sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc b/sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 b/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0
deleted file mode 100644
index da61769c6599d8b37b3c0038c0724860eab55094..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0
+++ /dev/null
@@ -1,22 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_CREATETABLE (TOK_TABNAME serde_regex1) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL key TOK_DECIMAL) (TOK_TABCOL value TOK_INT)) (TOK_TABLESERIALIZER (TOK_SERDENAME 'org.apache.hadoop.hive.serde2.RegexSerDe' (TOK_TABLEPROPERTIES (TOK_TABLEPROPLIST (TOK_TABLEPROPERTY "input.regex" "([^ ]*) ([^ ]*)"))))) TOK_TBLTEXTFILE)
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Create Table Operator:
-        Create Table
-          columns: key decimal, value int
-          if not exists: false
-          input format: org.apache.hadoop.mapred.TextInputFormat
-          # buckets: -1
-          output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
-          serde name: org.apache.hadoop.hive.serde2.RegexSerDe
-          serde properties:
-            input.regex ([^ ]*) ([^ ]*)
-          name: serde_regex1
-          isExternal: false
-
-
diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 b/sql/hive/src/test/resources/golden/serde_regex-7-bf456bcf6be7334488424dfeadf27d75
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2
rename to sql/hive/src/test/resources/golden/serde_regex-7-bf456bcf6be7334488424dfeadf27d75
diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 b/sql/hive/src/test/resources/golden/serde_regex-8-a4cf34af32b83e40e5c8b6d083938b54
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844
rename to sql/hive/src/test/resources/golden/serde_regex-8-a4cf34af32b83e40e5c8b6d083938b54
diff --git a/sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77 b/sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b b/sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 b/sql/hive/src/test/resources/golden/serde_regex-9-f1175f3322abec6f258dd49a5905bce0
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494
rename to sql/hive/src/test/resources/golden/serde_regex-9-f1175f3322abec6f258dd49a5905bce0
diff --git a/sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 b/sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1
index d1168556e09d43b99d4a44fb6da090bc7b17ee40..b1663e9a8c00db6306738c13ffe27d817a1a63be 100644
--- a/sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1
+++ b/sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1
@@ -1,11 +1,11 @@
 myint               	int                 	from deserializer   
 mystring            	string              	from deserializer   
 underscore_int      	int                 	from deserializer   
-b                   	string              	None                
+b                   	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-b                   	string              	None                
+b                   	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:int_string, dbName:default, owner:marmbrus, createTime:1389733035, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/int_string, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.IntString, serialization.format=org.apache.thrift.protocol.TBinaryProtocol}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:b, type:string, comment:null)], parameters:{transient_lastDdlTime=1389733035}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
\ No newline at end of file
+Detailed Table Information	Table(tableName:int_string, dbName:default, owner:marmbrus, createTime:1413891326, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/int_string, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.IntString, serialization.format=org.apache.thrift.protocol.TBinaryProtocol}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:b, type:string, comment:null)], parameters:{transient_lastDdlTime=1413891326}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa b/sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa
index 0b8f428b24193a6ef72a5e6a15cb81b298e2037e..f5ec7a9aca8f3677f42b6139e74e455d404d5406 100644
--- a/sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa
+++ b/sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa
@@ -1,11 +1,11 @@
 myint               	int                 	from deserializer   
 mystring            	string              	from deserializer   
 underscore_int      	int                 	from deserializer   
-b                   	string              	None                
+b                   	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-b                   	string              	None                
+b                   	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[part1], dbName:default, tableName:int_string, createTime:1389733036, lastAccessTime:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/int_string/b=part1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.IntString, serialization.format=org.apache.thrift.protocol.TBinaryProtocol}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1389733036})	
\ No newline at end of file
+Detailed Partition Information	Partition(values:[part1], dbName:default, tableName:int_string, createTime:1413891326, lastAccessTime:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/int_string/b=part1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.IntString, serialization.format=org.apache.thrift.protocol.TBinaryProtocol}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413891326})	
diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d b/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d
+++ b/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 b/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67
+++ b/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 b/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7
+++ b/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 b/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7
+++ b/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd b/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd
+++ b/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b b/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b
+++ b/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d b/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d
+++ b/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 b/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2
+++ b/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26
index 3c1fc128bedce6553e8c8ae0efeaded723d41e8e..69c7709aa90cb6559e849b2e489738628822635c 100644
--- a/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26
+++ b/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26
@@ -1,6 +1,6 @@
-CREATE EXTERNAL TABLE tmp_showcrt1(
-  key smallint, 
-  value float)
+CREATE EXTERNAL TABLE `tmp_showcrt1`(
+  `key` smallint, 
+  `value` float)
 CLUSTERED BY ( 
   key) 
 SORTED BY ( 
@@ -13,6 +13,6 @@ STORED AS INPUTFORMAT
 OUTPUTFORMAT 
   'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'
 LOCATION
-  'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1'
+  'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1'
 TBLPROPERTIES (
-  'transient_lastDdlTime'='1407132100')
+  'transient_lastDdlTime'='1413891329')
diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26
index 2ece813dd7d56f06105f44b0f382ceeade611bca..501bb6ab32f25e4aae2686be9fab25f71cd5d5d2 100644
--- a/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26
+++ b/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26
@@ -1,6 +1,6 @@
-CREATE  TABLE tmp_showcrt1(
-  key smallint, 
-  value float)
+CREATE  TABLE `tmp_showcrt1`(
+  `key` smallint, 
+  `value` float)
 COMMENT 'temporary table'
 CLUSTERED BY ( 
   key) 
@@ -14,9 +14,14 @@ STORED AS INPUTFORMAT
 OUTPUTFORMAT 
   'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'
 LOCATION
-  'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1'
+  'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1'
 TBLPROPERTIES (
+  'numFiles'='0', 
   'EXTERNAL'='FALSE', 
-  'last_modified_by'='tianyi', 
-  'last_modified_time'='1407132100', 
-  'transient_lastDdlTime'='1407132100')
+  'last_modified_by'='marmbrus', 
+  'last_modified_time'='1413891329', 
+  'transient_lastDdlTime'='1413891329', 
+  'COLUMN_STATS_ACCURATE'='false', 
+  'totalSize'='0', 
+  'numRows'='-1', 
+  'rawDataSize'='-1')
diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26
index 2af657bd295062794ee1022140f792adf0b32869..6e353675b5ed84a22c686b3632ec16f19db5f965 100644
--- a/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26
+++ b/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26
@@ -1,6 +1,6 @@
-CREATE EXTERNAL TABLE tmp_showcrt1(
-  key smallint, 
-  value float)
+CREATE EXTERNAL TABLE `tmp_showcrt1`(
+  `key` smallint, 
+  `value` float)
 COMMENT 'changed comment'
 CLUSTERED BY ( 
   key) 
@@ -14,8 +14,13 @@ STORED AS INPUTFORMAT
 OUTPUTFORMAT 
   'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'
 LOCATION
-  'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1'
+  'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1'
 TBLPROPERTIES (
-  'last_modified_by'='tianyi', 
-  'last_modified_time'='1407132100', 
-  'transient_lastDdlTime'='1407132100')
+  'numFiles'='0', 
+  'last_modified_by'='marmbrus', 
+  'last_modified_time'='1413891329', 
+  'transient_lastDdlTime'='1413891329', 
+  'COLUMN_STATS_ACCURATE'='false', 
+  'totalSize'='0', 
+  'numRows'='-1', 
+  'rawDataSize'='-1')
diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26
index f793ffb7a0bfdb0768a75b369385bc0147944679..6e353675b5ed84a22c686b3632ec16f19db5f965 100644
--- a/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26
+++ b/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26
@@ -1,6 +1,6 @@
-CREATE EXTERNAL TABLE tmp_showcrt1(
-  key smallint, 
-  value float)
+CREATE EXTERNAL TABLE `tmp_showcrt1`(
+  `key` smallint, 
+  `value` float)
 COMMENT 'changed comment'
 CLUSTERED BY ( 
   key) 
@@ -14,8 +14,13 @@ STORED AS INPUTFORMAT
 OUTPUTFORMAT 
   'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'
 LOCATION
-  'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1'
+  'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1'
 TBLPROPERTIES (
-  'last_modified_by'='tianyi', 
-  'last_modified_time'='1407132101', 
-  'transient_lastDdlTime'='1407132101')
+  'numFiles'='0', 
+  'last_modified_by'='marmbrus', 
+  'last_modified_time'='1413891329', 
+  'transient_lastDdlTime'='1413891329', 
+  'COLUMN_STATS_ACCURATE'='false', 
+  'totalSize'='0', 
+  'numRows'='-1', 
+  'rawDataSize'='-1')
diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26
index c65aff26a7fc11a74c4b146596f893c2a09944b6..da849512f4d8f60fbe9d9c48f882109b2f104224 100644
--- a/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26
+++ b/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26
@@ -1,6 +1,6 @@
-CREATE EXTERNAL TABLE tmp_showcrt1(
-  key smallint, 
-  value float)
+CREATE EXTERNAL TABLE `tmp_showcrt1`(
+  `key` smallint, 
+  `value` float)
 COMMENT 'changed comment'
 CLUSTERED BY ( 
   key) 
@@ -14,8 +14,13 @@ STORED BY
 WITH SERDEPROPERTIES ( 
   'serialization.format'='1')
 LOCATION
-  'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1'
+  'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1'
 TBLPROPERTIES (
-  'last_modified_by'='tianyi', 
-  'last_modified_time'='1407132101', 
-  'transient_lastDdlTime'='1407132101')
+  'numFiles'='0', 
+  'last_modified_by'='marmbrus', 
+  'last_modified_time'='1413891329', 
+  'transient_lastDdlTime'='1413891329', 
+  'COLUMN_STATS_ACCURATE'='false', 
+  'totalSize'='0', 
+  'numRows'='-1', 
+  'rawDataSize'='-1')
diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 b/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0
index b5a18368ed85e9587a411918767cc86b508be6cf..90f8415a1c6bea40f148d9ff79997cc72400b63b 100644
--- a/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0
+++ b/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0
@@ -1,6 +1,6 @@
-CREATE  TABLE tmp_feng.tmp_showcrt(
-  key string, 
-  value int)
+CREATE  TABLE `tmp_feng.tmp_showcrt`(
+  `key` string, 
+  `value` int)
 ROW FORMAT SERDE 
   'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' 
 STORED AS INPUTFORMAT 
@@ -8,6 +8,6 @@ STORED AS INPUTFORMAT
 OUTPUTFORMAT 
   'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'
 LOCATION
-  'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_feng.db/tmp_showcrt'
+  'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_feng.db/tmp_showcrt'
 TBLPROPERTIES (
-  'transient_lastDdlTime'='1407132107')
+  'transient_lastDdlTime'='1413891330')
diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-0-97228478b9925f06726ceebb6571bf34 b/sql/hive/src/test/resources/golden/show_create_table_delimited-0-97228478b9925f06726ceebb6571bf34
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26
index d36ad25dc82734e764e0790c6c6aae51b99086f1..4ee22e52303163bb8431b0a165da2e72e3045b94 100644
--- a/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26
+++ b/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26
@@ -1,7 +1,7 @@
-CREATE  TABLE tmp_showcrt1(
-  key int, 
-  value string, 
-  newvalue bigint)
+CREATE  TABLE `tmp_showcrt1`(
+  `key` int, 
+  `value` string, 
+  `newvalue` bigint)
 ROW FORMAT DELIMITED 
   FIELDS TERMINATED BY ',' 
   COLLECTION ITEMS TERMINATED BY '|' 
@@ -12,6 +12,6 @@ STORED AS INPUTFORMAT
 OUTPUTFORMAT 
   'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'
 LOCATION
-  'file:/tmp/tmp_showcrt1'
+  'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/testTempFiles4427612185729633290spark.hive.tmp/tmp_showcrt1'
 TBLPROPERTIES (
-  'transient_lastDdlTime'='1407132730')
+  'transient_lastDdlTime'='1413891331')
diff --git a/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26
index 9e572c0d7df6aa284e8852b6fb7aa48df6bf9848..2a1acca6efb8d5e222c7ca2e9cf887e5cb7fbbf9 100644
--- a/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26
+++ b/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26
@@ -1,9 +1,9 @@
-CREATE EXTERNAL TABLE tmp_showcrt1(
-  key string, 
-  newvalue boolean COMMENT 'a new value')
+CREATE EXTERNAL TABLE `tmp_showcrt1`(
+  `key` string, 
+  `newvalue` boolean COMMENT 'a new value')
 COMMENT 'temporary table'
 PARTITIONED BY ( 
-  value bigint COMMENT 'some value')
+  `value` bigint COMMENT 'some value')
 ROW FORMAT SERDE 
   'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' 
 STORED AS INPUTFORMAT 
@@ -11,6 +11,6 @@ STORED AS INPUTFORMAT
 OUTPUTFORMAT 
   'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'
 LOCATION
-  'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1'
+  'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1'
 TBLPROPERTIES (
-  'transient_lastDdlTime'='1407132112')
+  'transient_lastDdlTime'='1413891331')
diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26
index 69a38e1a7b20a90aa95f87d685fc97bc13550299..6fda2570b53f1e5820b4e823741bd9cb53bc861e 100644
--- a/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26
+++ b/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26
@@ -1,7 +1,7 @@
-CREATE  TABLE tmp_showcrt1(
-  key int, 
-  value string, 
-  newvalue bigint)
+CREATE  TABLE `tmp_showcrt1`(
+  `key` int, 
+  `value` string, 
+  `newvalue` bigint)
 COMMENT 'temporary table'
 ROW FORMAT SERDE 
   'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' 
@@ -10,6 +10,11 @@ STORED AS INPUTFORMAT
 OUTPUTFORMAT 
   'org.apache.hadoop.hive.ql.io.RCFileOutputFormat'
 LOCATION
-  'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1'
+  'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1'
 TBLPROPERTIES (
-  'transient_lastDdlTime'='1407132115')
+  'numFiles'='0', 
+  'transient_lastDdlTime'='1413891332', 
+  'COLUMN_STATS_ACCURATE'='false', 
+  'totalSize'='0', 
+  'numRows'='-1', 
+  'rawDataSize'='-1')
diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26
index b4e693dc622fb2e710844686d18a2505e482a193..cbbbb7b3ce3c72c9b168cf5130c41b38207d9e88 100644
--- a/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26
+++ b/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26
@@ -1,6 +1,6 @@
-CREATE EXTERNAL TABLE tmp_showcrt1(
-  key string, 
-  value boolean)
+CREATE EXTERNAL TABLE `tmp_showcrt1`(
+  `key` string, 
+  `value` boolean)
 ROW FORMAT SERDE 
   'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' 
 STORED BY 
@@ -9,6 +9,6 @@ WITH SERDEPROPERTIES (
   'serialization.format'='$', 
   'field.delim'=',')
 LOCATION
-  'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1'
+  'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1'
 TBLPROPERTIES (
-  'transient_lastDdlTime'='1407132115')
+  'transient_lastDdlTime'='1413891332')
diff --git a/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 b/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7
index be3fb3ce30960ed7e7bd15ab0fb7ef94406c2b1d..a721f07bb90b72f746268b3beddc47618d640401 100644
--- a/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7
+++ b/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7
@@ -1 +1 @@
-CREATE VIEW tmp_copy_src AS SELECT `src`.`key`, `src`.`value` FROM `default`.`src`
+CREATE VIEW `tmp_copy_src` AS SELECT `src`.`key`, `src`.`value` FROM `default`.`src`
diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 b/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780
index fd33cfcc9ab20d1ebb5bec6b03a331c3947e6307..442b05b4cf4ee3583e12ada2d9a2a65d05f2cc27 100644
--- a/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780
+++ b/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780
@@ -1 +1 @@
-concat
\ No newline at end of file
+concat
diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d b/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d
index fd33cfcc9ab20d1ebb5bec6b03a331c3947e6307..442b05b4cf4ee3583e12ada2d9a2a65d05f2cc27 100644
--- a/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d
+++ b/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d
@@ -1 +1 @@
-concat
\ No newline at end of file
+concat
diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 b/sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3
index a3998eeeee623a11185ceda5e93621e7c6a8a00e..b576089faa4846cba2061daa43cc9427dfce700d 100644
--- a/sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3
+++ b/sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3
@@ -1 +1 @@
-concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data  bin1, bin2, ... binN
\ No newline at end of file
+concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data  bin1, bin2, ... binN
diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 b/sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2
index a3998eeeee623a11185ceda5e93621e7c6a8a00e..b576089faa4846cba2061daa43cc9427dfce700d 100644
--- a/sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2
+++ b/sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2
@@ -1 +1 @@
-concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data  bin1, bin2, ... binN
\ No newline at end of file
+concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data  bin1, bin2, ... binN
diff --git a/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 b/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043
index 9d40ffaef5862ecb41df08fedee3f6fc1661b274..3049cd6243ad8eab127c297ff685dca6c913711c 100644
--- a/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043
+++ b/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043
@@ -32,6 +32,7 @@ case
 ceil
 ceiling
 coalesce
+collect_list
 collect_set
 compute_stats
 concat
@@ -45,6 +46,7 @@ covar_pop
 covar_samp
 create_union
 cume_dist
+current_database
 date_add
 date_sub
 datediff
@@ -123,6 +125,7 @@ percentile
 percentile_approx
 pi
 pmod
+posexplode
 positive
 pow
 power
@@ -189,4 +192,4 @@ xpath_short
 xpath_string
 year
 |
-~
\ No newline at end of file
+~
diff --git a/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 b/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797
index c62b965cb1559eaab7881b63f07effcece0a8306..175795534fff516ec8fc4ca407c75b7d076438ba 100644
--- a/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797
+++ b/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797
@@ -2,6 +2,7 @@ case
 ceil
 ceiling
 coalesce
+collect_list
 collect_set
 compute_stats
 concat
@@ -14,4 +15,5 @@ count
 covar_pop
 covar_samp
 create_union
-cume_dist
\ No newline at end of file
+cume_dist
+current_database
diff --git a/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c b/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c
index a56b5a3766c5ce23338350080a151301a83701a6..3c25d656bda1cb5b180e697d2ec6cbec42426b18 100644
--- a/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c
+++ b/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c
@@ -1,6 +1,7 @@
 assert_true
 case
 coalesce
+current_database
 decode
 e
 encode
@@ -19,6 +20,7 @@ negative
 ntile
 parse_url_tuple
 percentile
+posexplode
 positive
 regexp_replace
 reverse
@@ -29,4 +31,4 @@ to_date
 translate
 ucase
 variance
-xpath_double
\ No newline at end of file
+xpath_double
diff --git a/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 b/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049
index 86605075c3d25fb820f8ef2f314820d91f726129..b5a372a5ff50a00390562154ba99d3f1d550efa6 100644
--- a/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049
+++ b/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049
@@ -1,3 +1,3 @@
 log
 log10
-log2
\ No newline at end of file
+log2
diff --git a/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 b/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48
index 312f6cdbf68c723325bc6ca2142289272d946277..cd2e58d04a4ef0c859a397661fedbbcdb74d6056 100644
--- a/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48
+++ b/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48
@@ -1,4 +1,4 @@
 date_add
 date_sub
 datediff
-to_date
\ No newline at end of file
+to_date
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b
deleted file mode 100644
index e772f4a83fecdf170f16473def754baa135350d0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b
+++ /dev/null
@@ -1,16 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_SHOWINDEXES show_idx_full)
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Show Index Operator:
-        Show Indexes
-
-  Stage: Stage-1
-    Fetch Operator
-      limit: -1
-
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490
deleted file mode 100644
index d68fbfc9c1e0fec663ef9723a466572f35e1c352..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490
+++ /dev/null
@@ -1,4 +0,0 @@
-idx_1               	show_idx_full       	key                 	default__show_idx_full_idx_1__	compact             	
-idx_2               	show_idx_full       	value1              	default__show_idx_full_idx_2__	compact             	
-idx_comment         	show_idx_full       	value2              	default__show_idx_full_idx_comment__	compact             	index comment       
-idx_compound        	show_idx_full       	key, value1         	default__show_idx_full_idx_compound__	compact             	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e
deleted file mode 100644
index 94ee57be9dcaf68449b5098b7d19024391e8a821..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e
+++ /dev/null
@@ -1,16 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_SHOWINDEXES show_idx_empty)
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Show Index Operator:
-        Show Indexes
-
-  Stage: Stage-1
-    Fetch Operator
-      limit: -1
-
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec b/sql/hive/src/test/resources/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 b/sql/hive/src/test/resources/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec b/sql/hive/src/test/resources/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 b/sql/hive/src/test/resources/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 b/sql/hive/src/test/resources/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 b/sql/hive/src/test/resources/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7
deleted file mode 100644
index 7e68a8acb1a87767885929d8f71be064b76cc336..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7
+++ /dev/null
@@ -1,16 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_SHOWINDEXES show_idx_t1)
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Show Index Operator:
-        Show Indexes
-
-  Stage: Stage-1
-    Fetch Operator
-      limit: -1
-
diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 b/sql/hive/src/test/resources/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49
deleted file mode 100644
index 36d22451eba3e0b7a8d71cd86638c3f3773426ea..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49
+++ /dev/null
@@ -1 +0,0 @@
-idx_t1              	show_idx_t1         	key                 	default__show_idx_t1_idx_t1__	compact             	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 b/sql/hive/src/test/resources/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4
deleted file mode 100644
index 7e68a8acb1a87767885929d8f71be064b76cc336..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4
+++ /dev/null
@@ -1,16 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_SHOWINDEXES show_idx_t1)
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Show Index Operator:
-        Show Indexes
-
-  Stage: Stage-1
-    Fetch Operator
-      limit: -1
-
diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 b/sql/hive/src/test/resources/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58
deleted file mode 100644
index 36d22451eba3e0b7a8d71cd86638c3f3773426ea..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58
+++ /dev/null
@@ -1 +0,0 @@
-idx_t1              	show_idx_t1         	key                 	default__show_idx_t1_idx_t1__	compact             	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e b/sql/hive/src/test/resources/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e
deleted file mode 100644
index 4dddeee9a233f6c8e9b32a8cf6fe2fe1fc833050..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e
+++ /dev/null
@@ -1,16 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_SHOWINDEXES show_idx_t1 FORMATTED)
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Show Index Operator:
-        Show Indexes
-
-  Stage: Stage-1
-    Fetch Operator
-      limit: -1
-
diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 b/sql/hive/src/test/resources/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515
deleted file mode 100644
index 76e0434294b4f50d9b23599b69bacfafb63f09e2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515
+++ /dev/null
@@ -1,4 +0,0 @@
-idx_name            	tab_name            	col_names           	idx_tab_name        	idx_type            	comment             
-	 	 	 	 	 
-	 	 	 	 	 
-idx_t1              	show_idx_t1         	key                 	default__show_idx_t1_idx_t1__	compact             	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a b/sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a
index e9c723bbd136e561e0ff75d4b248e96ccbf81e73..8c43153cf66f9f265bbe39da3810288e57771358 100644
--- a/sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a
+++ b/sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a
@@ -1,4 +1,4 @@
 ds=2008-04-08/hr=11
 ds=2008-04-08/hr=12
 ds=2008-04-09/hr=11
-ds=2008-04-09/hr=12
\ No newline at end of file
+ds=2008-04-09/hr=12
diff --git a/sql/hive/src/test/resources/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/show_partitions-1-e69b801a3c6c5f6692050bcdb0e31db9
similarity index 100%
rename from sql/hive/src/test/resources/golden/stats4-2-463330cf55370dbe92d6ed74ef91302
rename to sql/hive/src/test/resources/golden/show_partitions-1-e69b801a3c6c5f6692050bcdb0e31db9
diff --git a/sql/hive/src/test/resources/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 b/sql/hive/src/test/resources/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0
deleted file mode 100644
index 19b4a62499762c99a55c9cab929e4168f50e50a9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0
+++ /dev/null
@@ -1,2 +0,0 @@
-ds=2008-04-08/hr=11
-ds=2008-04-09/hr=11
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca b/sql/hive/src/test/resources/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca
deleted file mode 100644
index f3614273fa8fa9d8c4a444539fa6fb4bfa8e9a09..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca
+++ /dev/null
@@ -1,2 +0,0 @@
-ds=2008-04-08/hr=11
-ds=2008-04-08/hr=12
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/show_partitions-2-e94d4100cb64c67f1127b4e255d28ae0 b/sql/hive/src/test/resources/golden/show_partitions-2-e94d4100cb64c67f1127b4e255d28ae0
new file mode 100644
index 0000000000000000000000000000000000000000..8b3fd053b6fb627d7fa6d3b897157798bd322791
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/show_partitions-2-e94d4100cb64c67f1127b4e255d28ae0
@@ -0,0 +1,2 @@
+ds=2008-04-08/hr=11
+ds=2008-04-09/hr=11
diff --git a/sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 b/sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70
deleted file mode 100644
index 0cdd3e8594c5991436a85d167f25fd33fc1198f2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70
+++ /dev/null
@@ -1 +0,0 @@
-ds=2008-04-08/hr=12
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 b/sql/hive/src/test/resources/golden/show_partitions-3-a1bde7c2c040b4d45ddceac9983c2ca
similarity index 100%
rename from sql/hive/src/test/resources/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32
rename to sql/hive/src/test/resources/golden/show_partitions-3-a1bde7c2c040b4d45ddceac9983c2ca
diff --git a/sql/hive/src/test/resources/golden/show_partitions-4-9e3f80cb6ed9883c715ea8aa3f391d70 b/sql/hive/src/test/resources/golden/show_partitions-4-9e3f80cb6ed9883c715ea8aa3f391d70
new file mode 100644
index 0000000000000000000000000000000000000000..dbd11ad78405bf850a88e363940c48419a52dbcb
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/show_partitions-4-9e3f80cb6ed9883c715ea8aa3f391d70
@@ -0,0 +1 @@
+ds=2008-04-08/hr=12
diff --git a/sql/hive/src/test/resources/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 b/sql/hive/src/test/resources/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce b/sql/hive/src/test/resources/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 b/sql/hive/src/test/resources/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tables-11-3f9a7f993510123059493826470f78f7 b/sql/hive/src/test/resources/golden/show_tables-11-3f9a7f993510123059493826470f78f7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c b/sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c
deleted file mode 100644
index 60f7943eda4a932339a5c69a19f22c8844201e3a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c
+++ /dev/null
@@ -1,3 +0,0 @@
-bar
-baz
-foo
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 b/sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657
deleted file mode 100644
index 4ffc580e2b8f38b2769d8ed5ab47a696f6c54be2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657
+++ /dev/null
@@ -1,4 +0,0 @@
-shtb_test1
-shtb_test2
-src
-srcpart
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c b/sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c
deleted file mode 100644
index 60f7943eda4a932339a5c69a19f22c8844201e3a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c
+++ /dev/null
@@ -1,3 +0,0 @@
-bar
-baz
-foo
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe b/sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe
deleted file mode 100644
index 4ffc580e2b8f38b2769d8ed5ab47a696f6c54be2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe
+++ /dev/null
@@ -1,4 +0,0 @@
-shtb_test1
-shtb_test2
-src
-srcpart
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 b/sql/hive/src/test/resources/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab b/sql/hive/src/test/resources/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 b/sql/hive/src/test/resources/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 b/sql/hive/src/test/resources/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 b/sql/hive/src/test/resources/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5
deleted file mode 100644
index 916ac1482c0618dbd92af4d4bd3c6baa10b402ee..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5
+++ /dev/null
@@ -1,18 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_SHOWTABLES 'shtb_*')
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Show Table Operator:
-        Show Tables
-          database name: default
-          pattern: shtb_*
-
-  Stage: Stage-1
-    Fetch Operator
-      limit: -1
-
diff --git a/sql/hive/src/test/resources/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 b/sql/hive/src/test/resources/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 b/sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391
deleted file mode 100644
index 19102815663d23f8b75a47e7a01965dcdc96468c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391
+++ /dev/null
@@ -1 +0,0 @@
-foo
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c b/sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c
deleted file mode 100644
index b67b816ee4b45abcb47534f403ab1f3f898bdd61..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c
+++ /dev/null
@@ -1,2 +0,0 @@
-shtb_test1
-shtb_test2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 b/sql/hive/src/test/resources/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35
deleted file mode 100644
index 8f06e234b2a6e027a4fecb0e89b3ff8de8946ede..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35
+++ /dev/null
@@ -1,18 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_SHOWTABLES 'shtb_test1|shtb_test2')
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Show Table Operator:
-        Show Tables
-          database name: default
-          pattern: shtb_test1|shtb_test2
-
-  Stage: Stage-1
-    Fetch Operator
-      limit: -1
-
diff --git a/sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d b/sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d
deleted file mode 100644
index b67b816ee4b45abcb47534f403ab1f3f898bdd61..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d
+++ /dev/null
@@ -1,2 +0,0 @@
-shtb_test1
-shtb_test2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 b/sql/hive/src/test/resources/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c b/sql/hive/src/test/resources/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 b/sql/hive/src/test/resources/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 b/sql/hive/src/test/resources/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa b/sql/hive/src/test/resources/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa
deleted file mode 100644
index bec424bb026e96de3d588d4ecff0a1c1240fe708..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa
+++ /dev/null
@@ -1,14 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_SHOW_TABLESTATUS `src` default)
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-
-  Stage: Stage-1
-    Fetch Operator
-      limit: -1
-
diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 b/sql/hive/src/test/resources/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7
deleted file mode 100644
index 9392b7dc686f6702eab4a3c8e4ef4a0c90c21098..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:src
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/src
-inputformat:org.apache.hadoop.mapred.TextInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-columns:struct columns { i32 key, string value}
-partitioned:false
-partitionColumns:
-totalNumberFiles:1
-totalFileSize:5812
-maxFileSize:5812
-minFileSize:5812
-lastAccessTime:0
-lastUpdateTime:1389733248000
diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 b/sql/hive/src/test/resources/golden/show_tablestatus-2-ecddce523f2af516700677a051581330
deleted file mode 100644
index 9392b7dc686f6702eab4a3c8e4ef4a0c90c21098..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tablestatus-2-ecddce523f2af516700677a051581330
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:src
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/src
-inputformat:org.apache.hadoop.mapred.TextInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-columns:struct columns { i32 key, string value}
-partitioned:false
-partitionColumns:
-totalNumberFiles:1
-totalFileSize:5812
-maxFileSize:5812
-minFileSize:5812
-lastAccessTime:0
-lastUpdateTime:1389733248000
diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d b/sql/hive/src/test/resources/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d
deleted file mode 100644
index 9392b7dc686f6702eab4a3c8e4ef4a0c90c21098..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:src
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/src
-inputformat:org.apache.hadoop.mapred.TextInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-columns:struct columns { i32 key, string value}
-partitioned:false
-partitionColumns:
-totalNumberFiles:1
-totalFileSize:5812
-maxFileSize:5812
-minFileSize:5812
-lastAccessTime:0
-lastUpdateTime:1389733248000
diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 b/sql/hive/src/test/resources/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 b/sql/hive/src/test/resources/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 b/sql/hive/src/test/resources/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c b/sql/hive/src/test/resources/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 b/sql/hive/src/test/resources/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51
deleted file mode 100644
index f8b64f6056ea7e8ad33a1338acb56f8ebe3eca67..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:srcpart
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/srcpart/ds=2008-04-08/hr=11
-inputformat:org.apache.hadoop.mapred.TextInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-columns:struct columns { i32 key, string value}
-partitioned:true
-partitionColumns:struct partition_columns { string ds, string hr}
-totalNumberFiles:1
-totalFileSize:5812
-maxFileSize:5812
-minFileSize:5812
-lastAccessTime:0
-lastUpdateTime:1389733249000
diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 b/sql/hive/src/test/resources/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9
deleted file mode 100644
index 9392b7dc686f6702eab4a3c8e4ef4a0c90c21098..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9
+++ /dev/null
@@ -1,14 +0,0 @@
-tableName:src
-owner:marmbrus
-location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/src
-inputformat:org.apache.hadoop.mapred.TextInputFormat
-outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-columns:struct columns { i32 key, string value}
-partitioned:false
-partitionColumns:
-totalNumberFiles:1
-totalFileSize:5812
-maxFileSize:5812
-minFileSize:5812
-lastAccessTime:0
-lastUpdateTime:1389733248000
diff --git a/sql/hive/src/test/resources/golden/show_tblproperties-4-6c63215ea599f6533666c4d70606b139 b/sql/hive/src/test/resources/golden/show_tblproperties-4-6c63215ea599f6533666c4d70606b139
index ce1a3441a1bc0ed02311288ddd4382276a985ecc..d882eea53ca3cb2636d2c8ff8fa78d15b5d9c26f 100644
--- a/sql/hive/src/test/resources/golden/show_tblproperties-4-6c63215ea599f6533666c4d70606b139
+++ b/sql/hive/src/test/resources/golden/show_tblproperties-4-6c63215ea599f6533666c4d70606b139
@@ -1,6 +1,10 @@
-	 
-last_modified_by	ocquery
-last_modified_time	1408598216
+numFiles	0
+last_modified_by	marmbrus
+last_modified_time	1413891337
 tmp	true
-transient_lastDdlTime	1408598216
+transient_lastDdlTime	1413891337
+COLUMN_STATS_ACCURATE	false
+totalSize	0
+numRows	-1
 bar	bar value
+rawDataSize	-1
diff --git a/sql/hive/src/test/resources/golden/showparts-0-593619bb962b318b82896658deaea1f1 b/sql/hive/src/test/resources/golden/showparts-0-593619bb962b318b82896658deaea1f1
deleted file mode 100644
index b590724bca78d74779cb33fb2118c05b373d3dc3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/showparts-0-593619bb962b318b82896658deaea1f1
+++ /dev/null
@@ -1,17 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_SHOWPARTITIONS srcpart)
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-      Show Partitions Operator:
-        Show Partitions
-          table: srcpart
-
-  Stage: Stage-1
-    Fetch Operator
-      limit: -1
-
diff --git a/sql/hive/src/test/resources/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a b/sql/hive/src/test/resources/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a
deleted file mode 100644
index e9c723bbd136e561e0ff75d4b248e96ccbf81e73..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a
+++ /dev/null
@@ -1,4 +0,0 @@
-ds=2008-04-08/hr=11
-ds=2008-04-08/hr=12
-ds=2008-04-09/hr=11
-ds=2008-04-09/hr=12
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 b/sql/hive/src/test/resources/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 b/sql/hive/src/test/resources/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 b/sql/hive/src/test/resources/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a b/sql/hive/src/test/resources/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 b/sql/hive/src/test/resources/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 b/sql/hive/src/test/resources/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 b/sql/hive/src/test/resources/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 b/sql/hive/src/test/resources/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 b/sql/hive/src/test/resources/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 b/sql/hive/src/test/resources/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd b/sql/hive/src/test/resources/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 b/sql/hive/src/test/resources/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557
+++ b/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22
+++ b/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 b/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223
index 872146532307a24150bfadd1bf63f5e2a41e14ed..19304c010452edd4780d6b69bb78cf1064888df4 100644
--- a/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223
+++ b/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223
@@ -1 +1 @@
-2	12	2	22	2	12
\ No newline at end of file
+2	12	2	22	2	12
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d
+++ b/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a b/sql/hive/src/test/resources/golden/skewjoinopt13-4-c0f14def6a135cc50cba364e810ce28e
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a
rename to sql/hive/src/test/resources/golden/skewjoinopt13-4-c0f14def6a135cc50cba364e810ce28e
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 b/sql/hive/src/test/resources/golden/skewjoinopt13-6-ade68a23d7b1a4f328623bb5a0f07488
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283
rename to sql/hive/src/test/resources/golden/skewjoinopt13-6-ade68a23d7b1a4f328623bb5a0f07488
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 b/sql/hive/src/test/resources/golden/skewjoinopt13-8-8eb53fb8f05a43ee377aa1c927857e7c
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080
rename to sql/hive/src/test/resources/golden/skewjoinopt13-8-8eb53fb8f05a43ee377aa1c927857e7c
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 b/sql/hive/src/test/resources/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 b/sql/hive/src/test/resources/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 b/sql/hive/src/test/resources/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 b/sql/hive/src/test/resources/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 b/sql/hive/src/test/resources/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 b/sql/hive/src/test/resources/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 b/sql/hive/src/test/resources/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 b/sql/hive/src/test/resources/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d b/sql/hive/src/test/resources/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 b/sql/hive/src/test/resources/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 b/sql/hive/src/test/resources/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e b/sql/hive/src/test/resources/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b b/sql/hive/src/test/resources/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 b/sql/hive/src/test/resources/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b b/sql/hive/src/test/resources/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 b/sql/hive/src/test/resources/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 b/sql/hive/src/test/resources/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557
+++ b/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22
+++ b/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa b/sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa
index ee1bb6b112381dc18029fb2638dfada9e01ffe74..6ca70c5267e65f2e04bf41f95fa2e22641b179d7 100644
--- a/sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa
+++ b/sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa
@@ -3,4 +3,4 @@
 8	18	8	18
 8	18	8	18
 8	28	8	18
-8	28	8	18
\ No newline at end of file
+8	28	8	18
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d
+++ b/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 b/sql/hive/src/test/resources/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a b/sql/hive/src/test/resources/golden/skewjoinopt18-4-abf4b7f158999af331d5dbfddf32fa68
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a
rename to sql/hive/src/test/resources/golden/skewjoinopt18-4-abf4b7f158999af331d5dbfddf32fa68
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 b/sql/hive/src/test/resources/golden/skewjoinopt18-8-ade68a23d7b1a4f328623bb5a0f07488
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6
rename to sql/hive/src/test/resources/golden/skewjoinopt18-8-ade68a23d7b1a4f328623bb5a0f07488
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e b/sql/hive/src/test/resources/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 b/sql/hive/src/test/resources/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d b/sql/hive/src/test/resources/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 b/sql/hive/src/test/resources/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 b/sql/hive/src/test/resources/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb b/sql/hive/src/test/resources/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 b/sql/hive/src/test/resources/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f b/sql/hive/src/test/resources/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 b/sql/hive/src/test/resources/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 b/sql/hive/src/test/resources/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b b/sql/hive/src/test/resources/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d b/sql/hive/src/test/resources/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 b/sql/hive/src/test/resources/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd b/sql/hive/src/test/resources/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 b/sql/hive/src/test/resources/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 b/sql/hive/src/test/resources/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e b/sql/hive/src/test/resources/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c b/sql/hive/src/test/resources/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 b/sql/hive/src/test/resources/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e b/sql/hive/src/test/resources/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd b/sql/hive/src/test/resources/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557
+++ b/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22
+++ b/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 b/sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8
index acd4039d356693932efe8051f87518206d5da47d..d3938a35d72f595a6fccd206830673caa921bc85 100644
--- a/sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8
+++ b/sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8
@@ -1,4 +1,4 @@
 2	1	2	22
 3	1	3	13
 8	2	8	18
-8	2	8	18
\ No newline at end of file
+8	2	8	18
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d
+++ b/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 b/sql/hive/src/test/resources/golden/skewjoinopt9-4-c0f14def6a135cc50cba364e810ce28e
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283
rename to sql/hive/src/test/resources/golden/skewjoinopt9-4-c0f14def6a135cc50cba364e810ce28e
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 b/sql/hive/src/test/resources/golden/skewjoinopt9-6-ade68a23d7b1a4f328623bb5a0f07488
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080
rename to sql/hive/src/test/resources/golden/skewjoinopt9-6-ade68a23d7b1a4f328623bb5a0f07488
diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d b/sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d
index f608d53f51ebe371818abc804df3a1e8352ae9bc..826338ea568104cd825ee1d18807a9c3b4b39698 100644
--- a/sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d
+++ b/sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d
@@ -9,4 +9,4 @@
 8	28	8	18
 8	28	8	18
 8	28	8	18
-8	28	8	18
\ No newline at end of file
+8	28	8	18
diff --git a/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 b/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619
index 7a442f02e8d7a78995089e1b3e97a71bb955daa3..1179e20c2847cba61d58c44c68c7b555f9b1b787 100644
--- a/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619
+++ b/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619
@@ -1,3 +1,3 @@
 0	2
 0	2
-0	2
\ No newline at end of file
+0	2
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244
index 836ee718649adc28166dbb9742ba03c242605bff..da83658b68646aa1d9b35fa4004800cd3e21eb72 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244
@@ -1 +1 @@
-51	val_3	51	val_30
\ No newline at end of file
+51	val_3	51	val_30
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa b/sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa
index 8f358bffec51d232c9b914e079255ea394f5cf32..1ab49661a01f4c2a2fe00a1d8e5273ea5b9187b7 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa
@@ -2,4 +2,4 @@
 51	val_3	51	val_30
 52	val_4	NULL	NULL
 53	val_5	NULL	NULL
-49	val_10	NULL	NULL
\ No newline at end of file
+49	val_10	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1
index 6197cf72c34542043af68c570ff92e8eadf3c397..ba7969b99d79fd7e99ac97761c242fd2db93f02d 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1
@@ -1,4 +1,4 @@
 NULL	NULL	50	val_20
 NULL	NULL	50	val_23
 NULL	NULL	50	val_25
-51	val_3	51	val_30
\ No newline at end of file
+51	val_3	51	val_30
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1
index d650d44f42404b7c3b4b0e62c7a7c9f4735a53af..ad8b511265e203e08b6cd7c76f7d70766eee6cd2 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1
@@ -5,4 +5,4 @@ NULL	NULL	50	val_23
 NULL	NULL	50	val_25
 51	val_3	51	val_30
 52	val_4	NULL	NULL
-53	val_5	NULL	NULL
\ No newline at end of file
+53	val_5	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1
index 836ee718649adc28166dbb9742ba03c242605bff..da83658b68646aa1d9b35fa4004800cd3e21eb72 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1
@@ -1 +1 @@
-51	val_3	51	val_30
\ No newline at end of file
+51	val_3	51	val_30
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838
index 8f358bffec51d232c9b914e079255ea394f5cf32..1ab49661a01f4c2a2fe00a1d8e5273ea5b9187b7 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838
@@ -2,4 +2,4 @@
 51	val_3	51	val_30
 52	val_4	NULL	NULL
 53	val_5	NULL	NULL
-49	val_10	NULL	NULL
\ No newline at end of file
+49	val_10	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b b/sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b
index 6197cf72c34542043af68c570ff92e8eadf3c397..ba7969b99d79fd7e99ac97761c242fd2db93f02d 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b
@@ -1,4 +1,4 @@
 NULL	NULL	50	val_20
 NULL	NULL	50	val_23
 NULL	NULL	50	val_25
-51	val_3	51	val_30
\ No newline at end of file
+51	val_3	51	val_30
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae b/sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae
index d650d44f42404b7c3b4b0e62c7a7c9f4735a53af..ad8b511265e203e08b6cd7c76f7d70766eee6cd2 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae
@@ -5,4 +5,4 @@ NULL	NULL	50	val_23
 NULL	NULL	50	val_25
 51	val_3	51	val_30
 52	val_4	NULL	NULL
-53	val_5	NULL	NULL
\ No newline at end of file
+53	val_5	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-3-bd7036a4c0b57349a588b974ffaa502
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5
rename to sql/hive/src/test/resources/golden/smb_mapjoin_1-3-bd7036a4c0b57349a588b974ffaa502
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-4-22ace1b9a0302d2b8a4aa57a2c2f6423
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785
rename to sql/hive/src/test/resources/golden/smb_mapjoin_1-4-22ace1b9a0302d2b8a4aa57a2c2f6423
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-5-6d835f651b099615df163be284e833de
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2
rename to sql/hive/src/test/resources/golden/smb_mapjoin_1-5-6d835f651b099615df163be284e833de
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-3-68d65d622e45f86d4a6c7d1d09ef823b
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390
rename to sql/hive/src/test/resources/golden/smb_mapjoin_10-3-68d65d622e45f86d4a6c7d1d09ef823b
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d31ad2289181131982ef3e9cd8c6386e
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2
rename to sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d31ad2289181131982ef3e9cd8c6386e
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b b/sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f7fb003fa65cadcd0b13cbdd7b355988
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b
rename to sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f7fb003fa65cadcd0b13cbdd7b355988
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-6-14b8b2e10032ab2d4a0e7a18979cdb59
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2
rename to sql/hive/src/test/resources/golden/smb_mapjoin_10-6-14b8b2e10032ab2d4a0e7a18979cdb59
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa b/sql/hive/src/test/resources/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af b/sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f b/sql/hive/src/test/resources/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb b/sql/hive/src/test/resources/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af b/sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af
deleted file mode 100644
index 8975db9a0503618f74bdeddba1ddf1f24119fb9c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af
+++ /dev/null
@@ -1 +0,0 @@
-293
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b b/sql/hive/src/test/resources/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e b/sql/hive/src/test/resources/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e b/sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e
deleted file mode 100644
index 8975db9a0503618f74bdeddba1ddf1f24119fb9c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e
+++ /dev/null
@@ -1 +0,0 @@
-293
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3
index b8280771579667f8d5d19f52730401750729b5da..9f4c46e548d04df5e8000a0f99bac01841bfcba8 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3
@@ -7,4 +7,4 @@
 0	val_0	0	val_0
 0	val_0	0	val_0
 0	val_0	0	val_0
-2	val_2	2	val_2
\ No newline at end of file
+2	val_2	2	val_2
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d b/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d
index 8fdd954df9831dfd29ceec0d74829b02f3f5d8c3..2bd5a0a98a36cc08ada88b804d3be047e6aa5b8a 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d
@@ -1 +1 @@
-22
\ No newline at end of file
+22
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a b/sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a
index 4a9735f855f96fbbe0a3a921080423139730a84b..ec7496a567609a3261094e21e78810dfb9095b44 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a
@@ -3,4 +3,4 @@
 4	1
 5	9
 8	1
-9	1
\ No newline at end of file
+9	1
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55
index 62f9457511f879886bb7728c986fe10b0ece6bcb..1e8b314962144c26d5e0e50fd29d2ca327864913 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55
@@ -1 +1 @@
-6
\ No newline at end of file
+6
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee b/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e b/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e
index 8fdd954df9831dfd29ceec0d74829b02f3f5d8c3..2bd5a0a98a36cc08ada88b804d3be047e6aa5b8a 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e
@@ -1 +1 @@
-22
\ No newline at end of file
+22
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b b/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070
index 2ebc6516c7df177c819b752dfac7b5ce5064189f..f6b91e0e1f8dddaac700b51aa7a66f29fc135a02 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070
@@ -1 +1 @@
-56
\ No newline at end of file
+56
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44
index 2edeafb09db0093bae6ff060e2dcd2166f5c9387..209e3ef4b6247ce746048d5711befda46206d235 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44
@@ -1 +1 @@
-20
\ No newline at end of file
+20
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a b/sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a
index b8280771579667f8d5d19f52730401750729b5da..9f4c46e548d04df5e8000a0f99bac01841bfcba8 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a
@@ -7,4 +7,4 @@
 0	val_0	0	val_0
 0	val_0	0	val_0
 0	val_0	0	val_0
-2	val_2	2	val_2
\ No newline at end of file
+2	val_2	2	val_2
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f b/sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f
index 471d725e7bfa313e867d063dafab3445b184dd7c..46d384b9dfb9b679bf473309ed5c1e3b9494a12f 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f
@@ -7,4 +7,4 @@
 0	0	val_0	0	0	val_0
 0	0	val_0	0	0	val_0
 0	0	val_0	0	0	val_0
-2	2	val_2	2	2	val_2
\ No newline at end of file
+2	2	val_2	2	2	val_2
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e b/sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e
index 471d725e7bfa313e867d063dafab3445b184dd7c..46d384b9dfb9b679bf473309ed5c1e3b9494a12f 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e
@@ -7,4 +7,4 @@
 0	0	val_0	0	0	val_0
 0	0	val_0	0	0	val_0
 0	0	val_0	0	0	val_0
-2	2	val_2	2	2	val_2
\ No newline at end of file
+2	2	val_2	2	2	val_2
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200
index 471d725e7bfa313e867d063dafab3445b184dd7c..46d384b9dfb9b679bf473309ed5c1e3b9494a12f 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200
@@ -7,4 +7,4 @@
 0	0	val_0	0	0	val_0
 0	0	val_0	0	0	val_0
 0	0	val_0	0	0	val_0
-2	2	val_2	2	2	val_2
\ No newline at end of file
+2	2	val_2	2	2	val_2
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf b/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf
index 67d892c80f49383b62e4332582a9483d07de7831..1f3d8a7a1fc08495c982b22af6f815d5955a9928 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf
@@ -1 +1 @@
-1028
\ No newline at end of file
+1028
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a b/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a
index 25398d9017c7f7e3489960d8e1daeb681a34ae6d..a84e60c846ab2d34272a777fb1c2cb95c60a7789 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a
@@ -1 +1 @@
-4378
\ No newline at end of file
+4378
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3
index 25398d9017c7f7e3489960d8e1daeb681a34ae6d..a84e60c846ab2d34272a777fb1c2cb95c60a7789 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3
@@ -1 +1 @@
-4378
\ No newline at end of file
+4378
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc b/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc
index 09b5b315bcf45481b8750788a2694318c181fc7f..e0fa4e2d8601d878b5d66a06c0eea51a2b5f1e7c 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc
@@ -1 +1 @@
-13126
\ No newline at end of file
+13126
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/sql/hive/src/test/resources/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f b/sql/hive/src/test/resources/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1
deleted file mode 100644
index eb1f49486af7c892e115f610fa32b505125766fc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1
+++ /dev/null
@@ -1 +0,0 @@
-500
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556
deleted file mode 100644
index 34251f6b242e79eb02dc1c4040ddbaa9fc924fce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556
+++ /dev/null
@@ -1 +0,0 @@
-247
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531
deleted file mode 100644
index 99bc3d518639f3433852c803ea7199085b6616cb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531
+++ /dev/null
@@ -1 +0,0 @@
-253
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504
deleted file mode 100644
index 34251f6b242e79eb02dc1c4040ddbaa9fc924fce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504
+++ /dev/null
@@ -1 +0,0 @@
-247
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156
deleted file mode 100644
index 99bc3d518639f3433852c803ea7199085b6616cb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156
+++ /dev/null
@@ -1 +0,0 @@
-253
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4
deleted file mode 100644
index eb1f49486af7c892e115f610fa32b505125766fc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4
+++ /dev/null
@@ -1 +0,0 @@
-500
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d b/sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d
deleted file mode 100644
index 34251f6b242e79eb02dc1c4040ddbaa9fc924fce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d
+++ /dev/null
@@ -1 +0,0 @@
-247
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5
deleted file mode 100644
index 99bc3d518639f3433852c803ea7199085b6616cb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5
+++ /dev/null
@@ -1 +0,0 @@
-253
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a b/sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a
deleted file mode 100644
index 34251f6b242e79eb02dc1c4040ddbaa9fc924fce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a
+++ /dev/null
@@ -1 +0,0 @@
-247
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264
deleted file mode 100644
index 99bc3d518639f3433852c803ea7199085b6616cb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264
+++ /dev/null
@@ -1 +0,0 @@
-253
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c b/sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c
deleted file mode 100644
index d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c
+++ /dev/null
@@ -1 +0,0 @@
-2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7
deleted file mode 100644
index d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7
+++ /dev/null
@@ -1 +0,0 @@
-2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f b/sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3
deleted file mode 100644
index d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3
+++ /dev/null
@@ -1 +0,0 @@
-2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab b/sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c b/sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe b/sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c b/sql/hive/src/test/resources/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/sql/hive/src/test/resources/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/sql/hive/src/test/resources/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1
deleted file mode 100644
index eb1f49486af7c892e115f610fa32b505125766fc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1
+++ /dev/null
@@ -1 +0,0 @@
-500
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19
deleted file mode 100644
index dce6588ca1420a69eea5699f2ec5d666daf16a6a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19
+++ /dev/null
@@ -1 +0,0 @@
-36
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06
deleted file mode 100644
index 86ee83a4a26867a79e0a5a8948153cff8f4cf9a4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06
+++ /dev/null
@@ -1 +0,0 @@
-40
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab b/sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab
deleted file mode 100644
index d99e90eb9675f72290ba32fbf844c1cb45c72718..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab
+++ /dev/null
@@ -1 +0,0 @@
-29
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc b/sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc
deleted file mode 100644
index dce6588ca1420a69eea5699f2ec5d666daf16a6a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc
+++ /dev/null
@@ -1 +0,0 @@
-36
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec b/sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec
deleted file mode 100644
index 86ee83a4a26867a79e0a5a8948153cff8f4cf9a4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec
+++ /dev/null
@@ -1 +0,0 @@
-40
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e b/sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e
deleted file mode 100644
index d99e90eb9675f72290ba32fbf844c1cb45c72718..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e
+++ /dev/null
@@ -1 +0,0 @@
-29
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4
deleted file mode 100644
index eb1f49486af7c892e115f610fa32b505125766fc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4
+++ /dev/null
@@ -1 +0,0 @@
-500
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b b/sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b
deleted file mode 100644
index dce6588ca1420a69eea5699f2ec5d666daf16a6a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b
+++ /dev/null
@@ -1 +0,0 @@
-36
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829
deleted file mode 100644
index 86ee83a4a26867a79e0a5a8948153cff8f4cf9a4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829
+++ /dev/null
@@ -1 +0,0 @@
-40
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f
deleted file mode 100644
index d99e90eb9675f72290ba32fbf844c1cb45c72718..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f
+++ /dev/null
@@ -1 +0,0 @@
-29
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb b/sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb
deleted file mode 100644
index dce6588ca1420a69eea5699f2ec5d666daf16a6a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb
+++ /dev/null
@@ -1 +0,0 @@
-36
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6
deleted file mode 100644
index 86ee83a4a26867a79e0a5a8948153cff8f4cf9a4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6
+++ /dev/null
@@ -1 +0,0 @@
-40
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f
deleted file mode 100644
index d99e90eb9675f72290ba32fbf844c1cb45c72718..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f
+++ /dev/null
@@ -1 +0,0 @@
-29
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d b/sql/hive/src/test/resources/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/sql/hive/src/test/resources/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac b/sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac
index 7b5974818c085c59b01e216c714d90ffd37dc1c2..4482f7ff91c62b314df8dde0a3cccb6d63c35c78 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac
@@ -4,4 +4,4 @@
 49	val_1	49	val_17
 49	val_10	49	val_17
 49	val_1	49	val_19
-49	val_10	49	val_19
\ No newline at end of file
+49	val_10	49	val_19
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89
index 0e1d132524064c070a10ba250ebd397d6d18c75a..cdacc0434caa7323b50574808064c2121302026c 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89
@@ -6,4 +6,4 @@
 53	val_5	NULL	NULL
 49	val_10	49	val_10
 49	val_10	49	val_17
-49	val_10	49	val_19
\ No newline at end of file
+49	val_10	49	val_19
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922
index 32be455fba8c45d454a000062902405d91b8f4ae..37d71f5b522c606b138f5fc4fb4ff9d0073ebacf 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922
@@ -6,4 +6,4 @@
 49	val_1	49	val_19
 49	val_10	49	val_19
 NULL	NULL	50	val_20
-NULL	NULL	50	val_23
\ No newline at end of file
+NULL	NULL	50	val_23
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca b/sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca
index 2496c4400b0b93ae533ac8244798ca5a2a09e4c0..364a70c242916a3729355ccb5b93ca152b55b7ac 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca
@@ -8,4 +8,4 @@ NULL	NULL	50	val_20
 NULL	NULL	50	val_23
 51	val_3	NULL	NULL
 52	val_4	52	val_4
-53	val_5	NULL	NULL
\ No newline at end of file
+53	val_5	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3
index 7b5974818c085c59b01e216c714d90ffd37dc1c2..4482f7ff91c62b314df8dde0a3cccb6d63c35c78 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3
@@ -4,4 +4,4 @@
 49	val_1	49	val_17
 49	val_10	49	val_17
 49	val_1	49	val_19
-49	val_10	49	val_19
\ No newline at end of file
+49	val_10	49	val_19
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b b/sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b
index 0e1d132524064c070a10ba250ebd397d6d18c75a..cdacc0434caa7323b50574808064c2121302026c 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b
@@ -6,4 +6,4 @@
 53	val_5	NULL	NULL
 49	val_10	49	val_10
 49	val_10	49	val_17
-49	val_10	49	val_19
\ No newline at end of file
+49	val_10	49	val_19
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84
index 32be455fba8c45d454a000062902405d91b8f4ae..37d71f5b522c606b138f5fc4fb4ff9d0073ebacf 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84
@@ -6,4 +6,4 @@
 49	val_1	49	val_19
 49	val_10	49	val_19
 NULL	NULL	50	val_20
-NULL	NULL	50	val_23
\ No newline at end of file
+NULL	NULL	50	val_23
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498
index 2496c4400b0b93ae533ac8244798ca5a2a09e4c0..364a70c242916a3729355ccb5b93ca152b55b7ac 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498
@@ -8,4 +8,4 @@ NULL	NULL	50	val_20
 NULL	NULL	50	val_23
 51	val_3	NULL	NULL
 52	val_4	52	val_4
-53	val_5	NULL	NULL
\ No newline at end of file
+53	val_5	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab b/sql/hive/src/test/resources/golden/smb_mapjoin_2-3-bd7036a4c0b57349a588b974ffaa502
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab
rename to sql/hive/src/test/resources/golden/smb_mapjoin_2-3-bd7036a4c0b57349a588b974ffaa502
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-4-22ace1b9a0302d2b8a4aa57a2c2f6423
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478
rename to sql/hive/src/test/resources/golden/smb_mapjoin_2-4-22ace1b9a0302d2b8a4aa57a2c2f6423
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-5-6d835f651b099615df163be284e833de
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78
rename to sql/hive/src/test/resources/golden/smb_mapjoin_2-5-6d835f651b099615df163be284e833de
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4
deleted file mode 100644
index eb1f49486af7c892e115f610fa32b505125766fc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4
+++ /dev/null
@@ -1 +0,0 @@
-500
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d b/sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d
deleted file mode 100644
index b6e27607fb529c10ffeb626c858f55206242edb7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d
+++ /dev/null
@@ -1 +0,0 @@
-242
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5
deleted file mode 100644
index ce83bd94b3310d442003750e2bf8e7f2e28da90a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5
+++ /dev/null
@@ -1 +0,0 @@
-258
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a b/sql/hive/src/test/resources/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a b/sql/hive/src/test/resources/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e b/sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e
deleted file mode 100644
index eb1f49486af7c892e115f610fa32b505125766fc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e
+++ /dev/null
@@ -1 +0,0 @@
-500
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462
deleted file mode 100644
index 34251f6b242e79eb02dc1c4040ddbaa9fc924fce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462
+++ /dev/null
@@ -1 +0,0 @@
-247
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845
deleted file mode 100644
index 99bc3d518639f3433852c803ea7199085b6616cb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845
+++ /dev/null
@@ -1 +0,0 @@
-253
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519
deleted file mode 100644
index 34251f6b242e79eb02dc1c4040ddbaa9fc924fce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519
+++ /dev/null
@@ -1 +0,0 @@
-247
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366
deleted file mode 100644
index 99bc3d518639f3433852c803ea7199085b6616cb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366
+++ /dev/null
@@ -1 +0,0 @@
-253
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff b/sql/hive/src/test/resources/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/sql/hive/src/test/resources/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a b/sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a
deleted file mode 100644
index eb1f49486af7c892e115f610fa32b505125766fc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a
+++ /dev/null
@@ -1 +0,0 @@
-500
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80
deleted file mode 100644
index 99bc3d518639f3433852c803ea7199085b6616cb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80
+++ /dev/null
@@ -1 +0,0 @@
-253
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4
deleted file mode 100644
index eb1f49486af7c892e115f610fa32b505125766fc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4
+++ /dev/null
@@ -1 +0,0 @@
-500
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307
deleted file mode 100644
index 99bc3d518639f3433852c803ea7199085b6616cb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307
+++ /dev/null
@@ -1 +0,0 @@
-253
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe b/sql/hive/src/test/resources/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b b/sql/hive/src/test/resources/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e b/sql/hive/src/test/resources/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad b/sql/hive/src/test/resources/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a b/sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a
deleted file mode 100644
index eb1f49486af7c892e115f610fa32b505125766fc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a
+++ /dev/null
@@ -1 +0,0 @@
-500
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80
deleted file mode 100644
index 99bc3d518639f3433852c803ea7199085b6616cb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80
+++ /dev/null
@@ -1 +0,0 @@
-253
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4
deleted file mode 100644
index eb1f49486af7c892e115f610fa32b505125766fc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4
+++ /dev/null
@@ -1 +0,0 @@
-500
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307
deleted file mode 100644
index 99bc3d518639f3433852c803ea7199085b6616cb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307
+++ /dev/null
@@ -1 +0,0 @@
-253
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b b/sql/hive/src/test/resources/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb b/sql/hive/src/test/resources/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad b/sql/hive/src/test/resources/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc b/sql/hive/src/test/resources/golden/smb_mapjoin_25-10-bd7036a4c0b57349a588b974ffaa502
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc
rename to sql/hive/src/test/resources/golden/smb_mapjoin_25-10-bd7036a4c0b57349a588b974ffaa502
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-11-22ace1b9a0302d2b8a4aa57a2c2f6423
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520
rename to sql/hive/src/test/resources/golden/smb_mapjoin_25-11-22ace1b9a0302d2b8a4aa57a2c2f6423
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f b/sql/hive/src/test/resources/golden/smb_mapjoin_25-12-6d835f651b099615df163be284e833de
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f
rename to sql/hive/src/test/resources/golden/smb_mapjoin_25-12-6d835f651b099615df163be284e833de
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef b/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f b/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d b/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff b/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79
index 2c05a72679b22f65c81ee9c454cb37996748f0ef..8aa583680ba515e1d2ee9fb34b7687914bef459a 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79
@@ -3,4 +3,4 @@
 50	val_25	50	val_20
 50	val_20	50	val_23
 50	val_23	50	val_23
-50	val_25	50	val_23
\ No newline at end of file
+50	val_25	50	val_23
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6
index 0d9bce83ea87c314114d7c11d1bfb60eb3ad832d..8f8addd11eabf72cf43511384e27db81a46d44a3 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6
@@ -4,4 +4,4 @@
 50	val_23	50	val_23
 50	val_25	50	val_20
 50	val_25	50	val_23
-51	val_30	NULL	NULL
\ No newline at end of file
+51	val_30	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e b/sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e
index d4c1adc92802d425ff749eb591051d24182c6041..c94cb185c6199fb8eba8c903cf67c308e8f55623 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e
@@ -7,4 +7,4 @@ NULL	NULL	49	val_19
 50	val_25	50	val_20
 50	val_20	50	val_23
 50	val_23	50	val_23
-50	val_25	50	val_23
\ No newline at end of file
+50	val_25	50	val_23
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a b/sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a
index b62eec8583c9225df3ac2bedb637b53a38e8ee3d..fb499272e90c510180eff679f903b56f0a06c718 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a
@@ -8,4 +8,4 @@ NULL	NULL	49	val_19
 50	val_25	50	val_20
 50	val_25	50	val_23
 51	val_30	NULL	NULL
-NULL	NULL	52	val_4
\ No newline at end of file
+NULL	NULL	52	val_4
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1
index 2c05a72679b22f65c81ee9c454cb37996748f0ef..8aa583680ba515e1d2ee9fb34b7687914bef459a 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1
@@ -3,4 +3,4 @@
 50	val_25	50	val_20
 50	val_20	50	val_23
 50	val_23	50	val_23
-50	val_25	50	val_23
\ No newline at end of file
+50	val_25	50	val_23
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd b/sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd
index 0d9bce83ea87c314114d7c11d1bfb60eb3ad832d..8f8addd11eabf72cf43511384e27db81a46d44a3 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd
@@ -4,4 +4,4 @@
 50	val_23	50	val_23
 50	val_25	50	val_20
 50	val_25	50	val_23
-51	val_30	NULL	NULL
\ No newline at end of file
+51	val_30	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06
index d4c1adc92802d425ff749eb591051d24182c6041..c94cb185c6199fb8eba8c903cf67c308e8f55623 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06
@@ -7,4 +7,4 @@ NULL	NULL	49	val_19
 50	val_25	50	val_20
 50	val_20	50	val_23
 50	val_23	50	val_23
-50	val_25	50	val_23
\ No newline at end of file
+50	val_25	50	val_23
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25
index b62eec8583c9225df3ac2bedb637b53a38e8ee3d..fb499272e90c510180eff679f903b56f0a06c718 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25
@@ -8,4 +8,4 @@ NULL	NULL	49	val_19
 50	val_25	50	val_20
 50	val_25	50	val_23
 51	val_30	NULL	NULL
-NULL	NULL	52	val_4
\ No newline at end of file
+NULL	NULL	52	val_4
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-3-bd7036a4c0b57349a588b974ffaa502
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288
rename to sql/hive/src/test/resources/golden/smb_mapjoin_3-3-bd7036a4c0b57349a588b974ffaa502
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-4-22ace1b9a0302d2b8a4aa57a2c2f6423
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799
rename to sql/hive/src/test/resources/golden/smb_mapjoin_3-4-22ace1b9a0302d2b8a4aa57a2c2f6423
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea b/sql/hive/src/test/resources/golden/smb_mapjoin_3-5-6d835f651b099615df163be284e833de
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea
rename to sql/hive/src/test/resources/golden/smb_mapjoin_3-5-6d835f651b099615df163be284e833de
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d b/sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d
index 46af2ffd2b6204513d70e08937e76584acb49b29..31c2549443bb19f7942394a2e34850e69ad3668d 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d
@@ -2,4 +2,4 @@
 51	val_3	51	val_30	NULL	NULL
 52	val_4	NULL	NULL	NULL	NULL
 53	val_5	NULL	NULL	NULL	NULL
-49	val_10	NULL	NULL	NULL	NULL
\ No newline at end of file
+49	val_10	NULL	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57
index 9047baeb236e5673329ebae184e36ec1c1ecfd95..fa5860b62f61107b540fd970d96382f2c27162d4 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57
@@ -3,4 +3,4 @@ NULL	NULL	NULL	NULL	49	val_10
 NULL	NULL	NULL	NULL	49	val_17
 NULL	NULL	NULL	NULL	49	val_19
 NULL	NULL	NULL	NULL	50	val_20
-NULL	NULL	NULL	NULL	50	val_23
\ No newline at end of file
+NULL	NULL	NULL	NULL	50	val_23
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5
index 47626758b41828eca5d509a85484a57a0fdbae4e..00e10b37e2fa1014262101e4e26229dd958d2ac1 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5
@@ -8,4 +8,4 @@ NULL	NULL	NULL	NULL	50	val_23
 51	val_3	51	val_30	NULL	NULL
 52	val_4	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	52	val_4
-53	val_5	NULL	NULL	NULL	NULL
\ No newline at end of file
+53	val_5	NULL	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09
index 29616d6368661d1c803915f9e1575847095ec3a9..84486137a37e41f4b03dbac817f96a22b2e813a1 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09
@@ -3,4 +3,4 @@ NULL	NULL	50	val_23	50	val_20
 NULL	NULL	50	val_25	50	val_20
 NULL	NULL	50	val_20	50	val_23
 NULL	NULL	50	val_23	50	val_23
-NULL	NULL	50	val_25	50	val_23
\ No newline at end of file
+NULL	NULL	50	val_25	50	val_23
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308
index 6f90cbf8396569c3fb4f0c92a0472d5c5d428be2..e2c6bab43530df454f5743c977452d70bd3ba827 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308
@@ -4,4 +4,4 @@ NULL	NULL	50	val_23	50	val_20
 NULL	NULL	50	val_23	50	val_23
 NULL	NULL	50	val_25	50	val_20
 NULL	NULL	50	val_25	50	val_23
-51	val_3	51	val_30	NULL	NULL
\ No newline at end of file
+51	val_3	51	val_30	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86
index 3b195f9dff935e84e115ff1182d6929f79bcb2ba..8ea9abf42f67845385f1a74382136e08c316d24e 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86
@@ -7,4 +7,4 @@ NULL	NULL	50	val_23	50	val_20
 NULL	NULL	50	val_25	50	val_20
 NULL	NULL	50	val_20	50	val_23
 NULL	NULL	50	val_23	50	val_23
-NULL	NULL	50	val_25	50	val_23
\ No newline at end of file
+NULL	NULL	50	val_25	50	val_23
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e b/sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e
index e18ae75d14a371eaffac1ea8975701adf71fb2ad..f0bda77df920226fa627454c480df5eec3b25b00 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e
@@ -8,4 +8,4 @@ NULL	NULL	50	val_23	50	val_23
 NULL	NULL	50	val_25	50	val_20
 NULL	NULL	50	val_25	50	val_23
 51	val_3	51	val_30	NULL	NULL
-NULL	NULL	NULL	NULL	52	val_4
\ No newline at end of file
+NULL	NULL	NULL	NULL	52	val_4
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54
index 8e51f0864314ae8eefffa3126b8c783060ed68e8..0c9e2268914bd0bd571a1e51e1ed70b7bd9fe982 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54
@@ -3,4 +3,4 @@ NULL	NULL	50	val_20	50	val_23
 NULL	NULL	50	val_23	50	val_20
 NULL	NULL	50	val_23	50	val_23
 NULL	NULL	50	val_25	50	val_20
-NULL	NULL	50	val_25	50	val_23
\ No newline at end of file
+NULL	NULL	50	val_25	50	val_23
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b b/sql/hive/src/test/resources/golden/smb_mapjoin_4-3-bd7036a4c0b57349a588b974ffaa502
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b
rename to sql/hive/src/test/resources/golden/smb_mapjoin_4-3-bd7036a4c0b57349a588b974ffaa502
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f b/sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f
index 51676ca2c94e4d0ef258bab145ff43ac34744856..c9ca4763cbabe0971391b85ff6f076630cbac13f 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f
@@ -8,4 +8,4 @@ NULL	NULL	50	val_25	50	val_20
 NULL	NULL	50	val_25	50	val_23
 51	val_3	51	val_30	NULL	NULL
 52	val_4	NULL	NULL	NULL	NULL
-53	val_5	NULL	NULL	NULL	NULL
\ No newline at end of file
+53	val_5	NULL	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8
index 4a125291de2715fe6e328728e3e83aeffe1f15d0..812d839a1c6423c9bbce26b064d5dca4c39e1819 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8
@@ -7,4 +7,4 @@ NULL	NULL	50	val_23	50	val_20
 NULL	NULL	50	val_23	50	val_23
 NULL	NULL	50	val_25	50	val_20
 NULL	NULL	50	val_25	50	val_23
-NULL	NULL	NULL	NULL	52	val_4
\ No newline at end of file
+NULL	NULL	NULL	NULL	52	val_4
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3
index 660e9044e889a20a0224398e3c26b3550e89d1ad..c019550d25827e223d18f6651b740f019b73d540 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3
@@ -12,4 +12,4 @@ NULL	NULL	50	val_25	50	val_23
 51	val_3	51	val_30	NULL	NULL
 52	val_4	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	52	val_4
-53	val_5	NULL	NULL	NULL	NULL
\ No newline at end of file
+53	val_5	NULL	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-4-22ace1b9a0302d2b8a4aa57a2c2f6423
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57
rename to sql/hive/src/test/resources/golden/smb_mapjoin_4-4-22ace1b9a0302d2b8a4aa57a2c2f6423
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-5-6d835f651b099615df163be284e833de
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447
rename to sql/hive/src/test/resources/golden/smb_mapjoin_4-5-6d835f651b099615df163be284e833de
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280
index 46af2ffd2b6204513d70e08937e76584acb49b29..31c2549443bb19f7942394a2e34850e69ad3668d 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280
@@ -2,4 +2,4 @@
 51	val_3	51	val_30	NULL	NULL
 52	val_4	NULL	NULL	NULL	NULL
 53	val_5	NULL	NULL	NULL	NULL
-49	val_10	NULL	NULL	NULL	NULL
\ No newline at end of file
+49	val_10	NULL	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9
index 9047baeb236e5673329ebae184e36ec1c1ecfd95..fa5860b62f61107b540fd970d96382f2c27162d4 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9
@@ -3,4 +3,4 @@ NULL	NULL	NULL	NULL	49	val_10
 NULL	NULL	NULL	NULL	49	val_17
 NULL	NULL	NULL	NULL	49	val_19
 NULL	NULL	NULL	NULL	50	val_20
-NULL	NULL	NULL	NULL	50	val_23
\ No newline at end of file
+NULL	NULL	NULL	NULL	50	val_23
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc b/sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc
index 47626758b41828eca5d509a85484a57a0fdbae4e..00e10b37e2fa1014262101e4e26229dd958d2ac1 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc
@@ -8,4 +8,4 @@ NULL	NULL	NULL	NULL	50	val_23
 51	val_3	51	val_30	NULL	NULL
 52	val_4	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	52	val_4
-53	val_5	NULL	NULL	NULL	NULL
\ No newline at end of file
+53	val_5	NULL	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d b/sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d
index 29616d6368661d1c803915f9e1575847095ec3a9..84486137a37e41f4b03dbac817f96a22b2e813a1 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d
@@ -3,4 +3,4 @@ NULL	NULL	50	val_23	50	val_20
 NULL	NULL	50	val_25	50	val_20
 NULL	NULL	50	val_20	50	val_23
 NULL	NULL	50	val_23	50	val_23
-NULL	NULL	50	val_25	50	val_23
\ No newline at end of file
+NULL	NULL	50	val_25	50	val_23
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a b/sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a
index 6f90cbf8396569c3fb4f0c92a0472d5c5d428be2..e2c6bab43530df454f5743c977452d70bd3ba827 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a
@@ -4,4 +4,4 @@ NULL	NULL	50	val_23	50	val_20
 NULL	NULL	50	val_23	50	val_23
 NULL	NULL	50	val_25	50	val_20
 NULL	NULL	50	val_25	50	val_23
-51	val_3	51	val_30	NULL	NULL
\ No newline at end of file
+51	val_3	51	val_30	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b b/sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b
index 3b195f9dff935e84e115ff1182d6929f79bcb2ba..8ea9abf42f67845385f1a74382136e08c316d24e 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b
@@ -7,4 +7,4 @@ NULL	NULL	50	val_23	50	val_20
 NULL	NULL	50	val_25	50	val_20
 NULL	NULL	50	val_20	50	val_23
 NULL	NULL	50	val_23	50	val_23
-NULL	NULL	50	val_25	50	val_23
\ No newline at end of file
+NULL	NULL	50	val_25	50	val_23
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177
index e18ae75d14a371eaffac1ea8975701adf71fb2ad..f0bda77df920226fa627454c480df5eec3b25b00 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177
@@ -8,4 +8,4 @@ NULL	NULL	50	val_23	50	val_23
 NULL	NULL	50	val_25	50	val_20
 NULL	NULL	50	val_25	50	val_23
 51	val_3	51	val_30	NULL	NULL
-NULL	NULL	NULL	NULL	52	val_4
\ No newline at end of file
+NULL	NULL	NULL	NULL	52	val_4
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7
index 8e51f0864314ae8eefffa3126b8c783060ed68e8..0c9e2268914bd0bd571a1e51e1ed70b7bd9fe982 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7
@@ -3,4 +3,4 @@ NULL	NULL	50	val_20	50	val_23
 NULL	NULL	50	val_23	50	val_20
 NULL	NULL	50	val_23	50	val_23
 NULL	NULL	50	val_25	50	val_20
-NULL	NULL	50	val_25	50	val_23
\ No newline at end of file
+NULL	NULL	50	val_25	50	val_23
diff --git a/sql/hive/src/test/resources/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-3-bd7036a4c0b57349a588b974ffaa502
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646
rename to sql/hive/src/test/resources/golden/smb_mapjoin_5-3-bd7036a4c0b57349a588b974ffaa502
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609
index 51676ca2c94e4d0ef258bab145ff43ac34744856..c9ca4763cbabe0971391b85ff6f076630cbac13f 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609
@@ -8,4 +8,4 @@ NULL	NULL	50	val_25	50	val_20
 NULL	NULL	50	val_25	50	val_23
 51	val_3	51	val_30	NULL	NULL
 52	val_4	NULL	NULL	NULL	NULL
-53	val_5	NULL	NULL	NULL	NULL
\ No newline at end of file
+53	val_5	NULL	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc b/sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc
index 4a125291de2715fe6e328728e3e83aeffe1f15d0..812d839a1c6423c9bbce26b064d5dca4c39e1819 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc
@@ -7,4 +7,4 @@ NULL	NULL	50	val_23	50	val_20
 NULL	NULL	50	val_23	50	val_23
 NULL	NULL	50	val_25	50	val_20
 NULL	NULL	50	val_25	50	val_23
-NULL	NULL	NULL	NULL	52	val_4
\ No newline at end of file
+NULL	NULL	NULL	NULL	52	val_4
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895
index 660e9044e889a20a0224398e3c26b3550e89d1ad..c019550d25827e223d18f6651b740f019b73d540 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895
@@ -12,4 +12,4 @@ NULL	NULL	50	val_25	50	val_23
 51	val_3	51	val_30	NULL	NULL
 52	val_4	NULL	NULL	NULL	NULL
 NULL	NULL	NULL	NULL	52	val_4
-53	val_5	NULL	NULL	NULL	NULL
\ No newline at end of file
+53	val_5	NULL	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-4-22ace1b9a0302d2b8a4aa57a2c2f6423
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396
rename to sql/hive/src/test/resources/golden/smb_mapjoin_5-4-22ace1b9a0302d2b8a4aa57a2c2f6423
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-5-6d835f651b099615df163be284e833de
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364
rename to sql/hive/src/test/resources/golden/smb_mapjoin_5-5-6d835f651b099615df163be284e833de
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca b/sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca
index c2e3ea8b0c8e2a5198fe64e3322be72886970306..b212e93a0a8c2795b471511b8d5c5aeae0af43e6 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca
@@ -1025,4 +1025,4 @@
 498	val_498	498	val_498
 498	val_498	498	val_498
 498	val_498	498	val_498
-498	val_498	498	val_498
\ No newline at end of file
+498	val_498	498	val_498
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e b/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e
index 0e17d179f4167188fc02a8aa4c2df6f20c21646b..11bd621866ba896c8603108b3c73dc08e728f89a 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e
@@ -1 +1 @@
-278697	278697	101852390308	101852390308
\ No newline at end of file
+278697	278697	101852390308	101852390308
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a b/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a
index 0e17d179f4167188fc02a8aa4c2df6f20c21646b..11bd621866ba896c8603108b3c73dc08e728f89a 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a
@@ -1 +1 @@
-278697	278697	101852390308	101852390308
\ No newline at end of file
+278697	278697	101852390308	101852390308
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca b/sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca
index c2e3ea8b0c8e2a5198fe64e3322be72886970306..b212e93a0a8c2795b471511b8d5c5aeae0af43e6 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca
@@ -1025,4 +1025,4 @@
 498	val_498	498	val_498
 498	val_498	498	val_498
 498	val_498	498	val_498
-498	val_498	498	val_498
\ No newline at end of file
+498	val_498	498	val_498
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e b/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e
index 0e17d179f4167188fc02a8aa4c2df6f20c21646b..11bd621866ba896c8603108b3c73dc08e728f89a 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e
@@ -1 +1 @@
-278697	278697	101852390308	101852390308
\ No newline at end of file
+278697	278697	101852390308	101852390308
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a b/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a
index 0e17d179f4167188fc02a8aa4c2df6f20c21646b..11bd621866ba896c8603108b3c73dc08e728f89a 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a
@@ -1 +1 @@
-278697	278697	101852390308	101852390308
\ No newline at end of file
+278697	278697	101852390308	101852390308
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80
index 4cd5eefea2a45eb284574e14a63985eb4cabf659..dfdc3444cc0728fe4d1e2494a7307f3a57acce1f 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80
@@ -497,4 +497,4 @@ NULL	NULL	496	val_496
 NULL	NULL	497	val_497
 NULL	NULL	498	val_498
 NULL	NULL	498	val_498
-NULL	NULL	498	val_498
\ No newline at end of file
+NULL	NULL	498	val_498
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a b/sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a
index 4cd5eefea2a45eb284574e14a63985eb4cabf659..dfdc3444cc0728fe4d1e2494a7307f3a57acce1f 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a
@@ -497,4 +497,4 @@ NULL	NULL	496	val_496
 NULL	NULL	497	val_497
 NULL	NULL	498	val_498
 NULL	NULL	498	val_498
-NULL	NULL	498	val_498
\ No newline at end of file
+NULL	NULL	498	val_498
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e b/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e
index ca97f45c90026c15d0f2889d0cb798d2f1a86bdb..2e711200bae289fa0aeed546b9e578ac68c6157d 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e
@@ -1 +1 @@
-0	130091	0	36210398070
\ No newline at end of file
+0	130091	0	36210398070
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a b/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a
index ca97f45c90026c15d0f2889d0cb798d2f1a86bdb..2e711200bae289fa0aeed546b9e578ac68c6157d 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a
@@ -1 +1 @@
-0	130091	0	36210398070
\ No newline at end of file
+0	130091	0	36210398070
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff b/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff
index ca97f45c90026c15d0f2889d0cb798d2f1a86bdb..2e711200bae289fa0aeed546b9e578ac68c6157d 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff
@@ -1 +1 @@
-0	130091	0	36210398070
\ No newline at end of file
+0	130091	0	36210398070
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb b/sql/hive/src/test/resources/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac b/sql/hive/src/test/resources/golden/smb_mapjoin_7-8-f983875c44b290b0884a22b6be6adc8
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac
rename to sql/hive/src/test/resources/golden/smb_mapjoin_7-8-f983875c44b290b0884a22b6be6adc8
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f b/sql/hive/src/test/resources/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c b/sql/hive/src/test/resources/golden/smb_mapjoin_7-9-84a394d962965e38593883742cc32c0d
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c
rename to sql/hive/src/test/resources/golden/smb_mapjoin_7-9-84a394d962965e38593883742cc32c0d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-5-eee18fc4192a4aa92a066eb66513be93
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014
rename to sql/hive/src/test/resources/golden/smb_mapjoin_8-5-eee18fc4192a4aa92a066eb66513be93
diff --git a/sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 b/sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45
index 55d948599907211562da45bf68242aa0462d556f..b70e127e82d056a30f56d6141cafa419ffef0184 100644
--- a/sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45
+++ b/sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45
@@ -497,4 +497,4 @@
 497	val_497
 498	val_498
 498	val_498
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89
index 8fdd954df9831dfd29ceec0d74829b02f3f5d8c3..2bd5a0a98a36cc08ada88b804d3be047e6aa5b8a 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89
@@ -1 +1 @@
-22
\ No newline at end of file
+22
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd
index 8fdd954df9831dfd29ceec0d74829b02f3f5d8c3..2bd5a0a98a36cc08ada88b804d3be047e6aa5b8a 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd
@@ -1 +1 @@
-22
\ No newline at end of file
+22
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd
index 8fdd954df9831dfd29ceec0d74829b02f3f5d8c3..2bd5a0a98a36cc08ada88b804d3be047e6aa5b8a 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd
@@ -1 +1 @@
-22
\ No newline at end of file
+22
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd
index 8fdd954df9831dfd29ceec0d74829b02f3f5d8c3..2bd5a0a98a36cc08ada88b804d3be047e6aa5b8a 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd
@@ -1 +1 @@
-22
\ No newline at end of file
+22
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a
index 67d892c80f49383b62e4332582a9483d07de7831..1f3d8a7a1fc08495c982b22af6f815d5955a9928 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a
@@ -1 +1 @@
-1028
\ No newline at end of file
+1028
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a
index 67d892c80f49383b62e4332582a9483d07de7831..1f3d8a7a1fc08495c982b22af6f815d5955a9928 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a
@@ -1 +1 @@
-1028
\ No newline at end of file
+1028
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571
+++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 b/sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9
index 688eea009d2927bcf97095cc9200bffa4686a6c9..d3a7b34f283b1a6a6c6395b3c28ca5eea842eeb5 100644
--- a/sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9
+++ b/sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9
@@ -1,10 +1,10 @@
-key                 	string              	None                
-value               	string              	None                
-ds                  	string              	None                
+key                 	string              	                    
+value               	string              	                    
+ds                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
+ds                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[1], dbName:default, tableName:stats_partitioned, createTime:1390903702, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_partitioned/ds=1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390903702, numRows=500, totalSize=5812, rawDataSize=5312})	
+Detailed Partition Information	Partition(values:[1], dbName:default, tableName:stats_partitioned, createTime:1413893941, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_partitioned/ds=1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413893941, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=500, rawDataSize=5312})	
diff --git a/sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c b/sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c
index 4d3a3d8f237b32c3df77683198a8f65280a15bf7..965913df89c646be7225d9f84f8dc05e172ceb3a 100644
--- a/sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c
+++ b/sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c
@@ -1,10 +1,10 @@
-key                 	string              	None                
-value               	string              	None                
-ds                  	string              	None                
+key                 	string              	                    
+value               	string              	                    
+ds                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
+ds                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:stats_partitioned, dbName:default, owner:marmbrus, createTime:1390903694, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, p3=v3, transient_lastDdlTime=1390903702, numRows=500, totalSize=5812, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:stats_partitioned, dbName:default, owner:marmbrus, createTime:1413893934, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1413893934}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c b/sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c
index a18ab93992cc5c61b7dbd52d3b70c3e0eba651cb..b34342d7815e1c3f41b1e4c7a989586f18511612 100644
--- a/sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c
+++ b/sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c
@@ -1,4 +1,4 @@
-key                 	string              	None                
-value               	string              	None                
+key                 	string              	                    
+value               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:stats_non_partitioned, dbName:default, owner:marmbrus, createTime:1390903702, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_non_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numPartitions=0, numFiles=1, p3=v3, transient_lastDdlTime=1390903710, numRows=500, totalSize=5812, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:stats_non_partitioned, dbName:default, owner:marmbrus, createTime:1413893941, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_non_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=1, transient_lastDdlTime=1413893948, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=500, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 b/sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9
index d861060f13caebd98b7038ed1f6ab32c69dc672d..c1e56cd3180cbc1aa7c70f205d8a471a9dff3f57 100644
--- a/sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9
+++ b/sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9
@@ -1,10 +1,10 @@
-key                 	string              	None                
-value               	string              	None                
-ds                  	string              	None                
+key                 	string              	                    
+value               	string              	                    
+ds                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
+ds                  	string              	                    
 	 	 
-Detailed Partition Information	Partition(values:[1], dbName:default, tableName:stats_partitioned, createTime:1390903719, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_partitioned/ds=1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390903719, numRows=500, totalSize=5812, rawDataSize=5312})	
+Detailed Partition Information	Partition(values:[1], dbName:default, tableName:stats_partitioned, createTime:1413893955, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_partitioned/ds=1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413893955, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=500, rawDataSize=5312})	
diff --git a/sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c b/sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c
index aa4dfe1e5ba397de718b3606d4936ad8730fb934..25d227999c390abcef3bd05e3b1402b40b4a635a 100644
--- a/sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c
+++ b/sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c
@@ -1,10 +1,10 @@
-key                 	string              	None                
-value               	string              	None                
-ds                  	string              	None                
+key                 	string              	                    
+value               	string              	                    
+ds                  	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-ds                  	string              	None                
+ds                  	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:stats_partitioned, dbName:default, owner:marmbrus, createTime:1390903710, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, p3=v3, transient_lastDdlTime=1390903719, numRows=500, totalSize=5812, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:stats_partitioned, dbName:default, owner:marmbrus, createTime:1413893948, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1413893948}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c b/sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c
index 8a5ae7cda1c577e445758759fba39bcfdaa8877c..8b503b045f6ecb5a57103bca0082ab23756c3d71 100644
--- a/sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c
+++ b/sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c
@@ -1,4 +1,4 @@
-key                 	string              	None                
-value               	string              	None                
+key                 	string              	                    
+value               	string              	                    
 	 	 
-Detailed Table Information	Table(tableName:stats_non_partitioned, dbName:default, owner:marmbrus, createTime:1390903686, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_non_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numPartitions=0, numFiles=1, p3=v3, transient_lastDdlTime=1390903694, numRows=500, totalSize=5812, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
+Detailed Table Information	Table(tableName:stats_non_partitioned, dbName:default, owner:marmbrus, createTime:1413893927, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_non_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=1, transient_lastDdlTime=1413893934, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=500, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)	
diff --git a/sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e b/sql/hive/src/test/resources/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b b/sql/hive/src/test/resources/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b
deleted file mode 100644
index f87b67cb2006f7087ad7b12812afacfd7ed34c1f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b
+++ /dev/null
@@ -1,39 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 28 02:17:02 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_t1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	4                   
-	numPartitions       	4                   
-	numRows             	2000                
-	p3                  	v3                  
-	rawDataSize         	21248               
-	totalSize           	23248               
-	transient_lastDdlTime	1390904237          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 b/sql/hive/src/test/resources/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 b/sql/hive/src/test/resources/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 b/sql/hive/src/test/resources/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats2-8-72621dba638b15d244850018e9f64d7 b/sql/hive/src/test/resources/golden/stats2-8-72621dba638b15d244850018e9f64d7
deleted file mode 100644
index 19a5e1e042ae7c8b2e4c25e6a3d68936507f4144..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats2-8-72621dba638b15d244850018e9f64d7
+++ /dev/null
@@ -1,34 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 28 02:17:02 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_t1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	p3                  	v3                  
-	transient_lastDdlTime	1390904222          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e b/sql/hive/src/test/resources/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e
deleted file mode 100644
index 7ddea146475d1e508bf283b6bdc59bfd9bd8a23f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e
+++ /dev/null
@@ -1,19 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_t1) (TOK_PARTSPEC (TOK_PARTVAL ds) (TOK_PARTVAL hr))))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 depends on stages: Stage-0
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        analyze_t1 
-          TableScan
-            alias: analyze_t1
-
-  Stage: Stage-1
-    Stats-Aggr Operator
-
-
diff --git a/sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 b/sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b b/sql/hive/src/test/resources/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 b/sql/hive/src/test/resources/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats20-5-76509775cfe11bb98ee088188a07668a b/sql/hive/src/test/resources/golden/stats20-5-76509775cfe11bb98ee088188a07668a
deleted file mode 100644
index 5c8878cba5e71493bb176261d6dacb8822c05e8b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats20-5-76509775cfe11bb98ee088188a07668a
+++ /dev/null
@@ -1,37 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 28 15:57:53 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2617911713640090101/stats_partitioned	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	1                   
-	numPartitions       	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390953481          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 b/sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 b/sql/hive/src/test/resources/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats20-8-300c971de74642118d36d36349bc81aa b/sql/hive/src/test/resources/golden/stats20-8-300c971de74642118d36d36349bc81aa
deleted file mode 100644
index d9ba3814de367dfd9601c208702bfd4c399c2a0a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats20-8-300c971de74642118d36d36349bc81aa
+++ /dev/null
@@ -1,37 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 28 15:57:53 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2617911713640090101/stats_partitioned	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	1                   
-	numPartitions       	1                   
-	numRows             	500                 
-	rawDataSize         	0                   
-	totalSize           	5812                
-	transient_lastDdlTime	1390953492          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 b/sql/hive/src/test/resources/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9
deleted file mode 100644
index bf2a7f452edc3b08fdad3f9f8cafc91ede4a2534..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9
+++ /dev/null
@@ -1,6 +0,0 @@
-1	test_part	test_Part
-2	test_part	test_Part
-3	test_part	test_Part
-4	test_part	test_Part
-5	test_part	test_Part
-6	test_part	test_Part
diff --git a/sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa b/sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa
deleted file mode 100644
index 1e8b314962144c26d5e0e50fd29d2ca327864913..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa
+++ /dev/null
@@ -1 +0,0 @@
-6
diff --git a/sql/hive/src/test/resources/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc b/sql/hive/src/test/resources/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 b/sql/hive/src/test/resources/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa b/sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa
deleted file mode 100644
index 1e8b314962144c26d5e0e50fd29d2ca327864913..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa
+++ /dev/null
@@ -1 +0,0 @@
-6
diff --git a/sql/hive/src/test/resources/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 b/sql/hive/src/test/resources/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4
deleted file mode 100644
index bf2a7f452edc3b08fdad3f9f8cafc91ede4a2534..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4
+++ /dev/null
@@ -1,6 +0,0 @@
-1	test_part	test_Part
-2	test_part	test_Part
-3	test_part	test_Part
-4	test_part	test_Part
-5	test_part	test_Part
-6	test_part	test_Part
diff --git a/sql/hive/src/test/resources/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 b/sql/hive/src/test/resources/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 b/sql/hive/src/test/resources/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b b/sql/hive/src/test/resources/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b
deleted file mode 100644
index cb1c9def371dff49ab3d0bbc31d0dce16f2b12c7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b
+++ /dev/null
@@ -1,38 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-col1                	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-pcol1               	string              	None                
-pcol2               	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 28 02:17:38 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/hive_test_dst	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	1                   
-	numPartitions       	1                   
-	numRows             	6                   
-	p3                  	v3                  
-	rawDataSize         	6                   
-	totalSize           	171                 
-	transient_lastDdlTime	1390904285          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.SequenceFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 b/sql/hive/src/test/resources/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 b/sql/hive/src/test/resources/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats3-20-ca048ad81b3df7159822073d206f0790 b/sql/hive/src/test/resources/golden/stats3-20-ca048ad81b3df7159822073d206f0790
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats3-3-ca048ad81b3df7159822073d206f0790 b/sql/hive/src/test/resources/golden/stats3-3-ca048ad81b3df7159822073d206f0790
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 b/sql/hive/src/test/resources/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 b/sql/hive/src/test/resources/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1
deleted file mode 100644
index 8bc7c145744971f67f3a3ae5ba13e5c8ae0a171f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1
+++ /dev/null
@@ -1,43 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_LOAD '/Users/marmbrus/workspace/hive/data/files/test.dat' (TOK_TAB (TOK_TABNAME hive_test_src)) local overwrite)
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 depends on stages: Stage-0
-  Stage-2 depends on stages: Stage-1
-
-STAGE PLANS:
-  Stage: Stage-0
-    Copy
-      source: file:/Users/marmbrus/workspace/hive/data/files/test.dat
-      destination: file:/tmp/hive-marmbrus/hive_2014-01-28_02-17-38_072_1020656193356018580-1/-ext-10000
-
-  Stage: Stage-1
-    Move Operator
-      tables:
-          replace: true
-          source: file:/tmp/hive-marmbrus/hive_2014-01-28_02-17-38_072_1020656193356018580-1/-ext-10000
-          table:
-              input format: org.apache.hadoop.mapred.TextInputFormat
-              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-              properties:
-                bucket_count -1
-                columns col1
-                columns.types string
-                file.inputformat org.apache.hadoop.mapred.TextInputFormat
-                file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/hive_test_src
-                name default.hive_test_src
-                p3 v3
-                serialization.ddl struct hive_test_src { string col1}
-                serialization.format 1
-                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                transient_lastDdlTime 1390904258
-              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-              name: default.hive_test_src
-          tmp directory: file:/tmp/hive-marmbrus/hive_2014-01-28_02-17-38_072_1020656193356018580-1/-ext-10001
-
-  Stage: Stage-2
-    Stats-Aggr Operator
-
-
diff --git a/sql/hive/src/test/resources/golden/stats3-6-4bf1504274319c44d370b58092fe016c b/sql/hive/src/test/resources/golden/stats3-6-4bf1504274319c44d370b58092fe016c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats3-7-73d7d55d6e5a57aacce8618902904d b/sql/hive/src/test/resources/golden/stats3-7-73d7d55d6e5a57aacce8618902904d
deleted file mode 100644
index 4916ef91b086624a7500a4f3f9e74731f7768441..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats3-7-73d7d55d6e5a57aacce8618902904d
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-col1                	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 28 02:17:38 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/hive_test_src	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	1                   
-	numPartitions       	0                   
-	numRows             	0                   
-	p3                  	v3                  
-	rawDataSize         	0                   
-	totalSize           	11                  
-	transient_lastDdlTime	1390904258          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 b/sql/hive/src/test/resources/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 b/sql/hive/src/test/resources/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 b/sql/hive/src/test/resources/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617 b/sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617
deleted file mode 100644
index 1634adfc4d70d083df5abc5abbfd0853b23062c0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617
+++ /dev/null
@@ -1,2 +0,0 @@
-ds=2008-12-31/hr=11
-ds=2008-12-31/hr=12
diff --git a/sql/hive/src/test/resources/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 b/sql/hive/src/test/resources/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66
deleted file mode 100644
index d7a8f25b41301ec8d6a1bd3d230e31a65586444e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66
+++ /dev/null
@@ -1,1000 +0,0 @@
-238	val_238	2008-04-08	11
-86	val_86	2008-04-08	11
-311	val_311	2008-04-08	11
-27	val_27	2008-04-08	11
-165	val_165	2008-04-08	11
-409	val_409	2008-04-08	11
-255	val_255	2008-04-08	11
-278	val_278	2008-04-08	11
-98	val_98	2008-04-08	11
-484	val_484	2008-04-08	11
-265	val_265	2008-04-08	11
-193	val_193	2008-04-08	11
-401	val_401	2008-04-08	11
-150	val_150	2008-04-08	11
-273	val_273	2008-04-08	11
-224	val_224	2008-04-08	11
-369	val_369	2008-04-08	11
-66	val_66	2008-04-08	11
-128	val_128	2008-04-08	11
-213	val_213	2008-04-08	11
-146	val_146	2008-04-08	11
-406	val_406	2008-04-08	11
-429	val_429	2008-04-08	11
-374	val_374	2008-04-08	11
-152	val_152	2008-04-08	11
-469	val_469	2008-04-08	11
-145	val_145	2008-04-08	11
-495	val_495	2008-04-08	11
-37	val_37	2008-04-08	11
-327	val_327	2008-04-08	11
-281	val_281	2008-04-08	11
-277	val_277	2008-04-08	11
-209	val_209	2008-04-08	11
-15	val_15	2008-04-08	11
-82	val_82	2008-04-08	11
-403	val_403	2008-04-08	11
-166	val_166	2008-04-08	11
-417	val_417	2008-04-08	11
-430	val_430	2008-04-08	11
-252	val_252	2008-04-08	11
-292	val_292	2008-04-08	11
-219	val_219	2008-04-08	11
-287	val_287	2008-04-08	11
-153	val_153	2008-04-08	11
-193	val_193	2008-04-08	11
-338	val_338	2008-04-08	11
-446	val_446	2008-04-08	11
-459	val_459	2008-04-08	11
-394	val_394	2008-04-08	11
-237	val_237	2008-04-08	11
-482	val_482	2008-04-08	11
-174	val_174	2008-04-08	11
-413	val_413	2008-04-08	11
-494	val_494	2008-04-08	11
-207	val_207	2008-04-08	11
-199	val_199	2008-04-08	11
-466	val_466	2008-04-08	11
-208	val_208	2008-04-08	11
-174	val_174	2008-04-08	11
-399	val_399	2008-04-08	11
-396	val_396	2008-04-08	11
-247	val_247	2008-04-08	11
-417	val_417	2008-04-08	11
-489	val_489	2008-04-08	11
-162	val_162	2008-04-08	11
-377	val_377	2008-04-08	11
-397	val_397	2008-04-08	11
-309	val_309	2008-04-08	11
-365	val_365	2008-04-08	11
-266	val_266	2008-04-08	11
-439	val_439	2008-04-08	11
-342	val_342	2008-04-08	11
-367	val_367	2008-04-08	11
-325	val_325	2008-04-08	11
-167	val_167	2008-04-08	11
-195	val_195	2008-04-08	11
-475	val_475	2008-04-08	11
-17	val_17	2008-04-08	11
-113	val_113	2008-04-08	11
-155	val_155	2008-04-08	11
-203	val_203	2008-04-08	11
-339	val_339	2008-04-08	11
-0	val_0	2008-04-08	11
-455	val_455	2008-04-08	11
-128	val_128	2008-04-08	11
-311	val_311	2008-04-08	11
-316	val_316	2008-04-08	11
-57	val_57	2008-04-08	11
-302	val_302	2008-04-08	11
-205	val_205	2008-04-08	11
-149	val_149	2008-04-08	11
-438	val_438	2008-04-08	11
-345	val_345	2008-04-08	11
-129	val_129	2008-04-08	11
-170	val_170	2008-04-08	11
-20	val_20	2008-04-08	11
-489	val_489	2008-04-08	11
-157	val_157	2008-04-08	11
-378	val_378	2008-04-08	11
-221	val_221	2008-04-08	11
-92	val_92	2008-04-08	11
-111	val_111	2008-04-08	11
-47	val_47	2008-04-08	11
-72	val_72	2008-04-08	11
-4	val_4	2008-04-08	11
-280	val_280	2008-04-08	11
-35	val_35	2008-04-08	11
-427	val_427	2008-04-08	11
-277	val_277	2008-04-08	11
-208	val_208	2008-04-08	11
-356	val_356	2008-04-08	11
-399	val_399	2008-04-08	11
-169	val_169	2008-04-08	11
-382	val_382	2008-04-08	11
-498	val_498	2008-04-08	11
-125	val_125	2008-04-08	11
-386	val_386	2008-04-08	11
-437	val_437	2008-04-08	11
-469	val_469	2008-04-08	11
-192	val_192	2008-04-08	11
-286	val_286	2008-04-08	11
-187	val_187	2008-04-08	11
-176	val_176	2008-04-08	11
-54	val_54	2008-04-08	11
-459	val_459	2008-04-08	11
-51	val_51	2008-04-08	11
-138	val_138	2008-04-08	11
-103	val_103	2008-04-08	11
-239	val_239	2008-04-08	11
-213	val_213	2008-04-08	11
-216	val_216	2008-04-08	11
-430	val_430	2008-04-08	11
-278	val_278	2008-04-08	11
-176	val_176	2008-04-08	11
-289	val_289	2008-04-08	11
-221	val_221	2008-04-08	11
-65	val_65	2008-04-08	11
-318	val_318	2008-04-08	11
-332	val_332	2008-04-08	11
-311	val_311	2008-04-08	11
-275	val_275	2008-04-08	11
-137	val_137	2008-04-08	11
-241	val_241	2008-04-08	11
-83	val_83	2008-04-08	11
-333	val_333	2008-04-08	11
-180	val_180	2008-04-08	11
-284	val_284	2008-04-08	11
-12	val_12	2008-04-08	11
-230	val_230	2008-04-08	11
-181	val_181	2008-04-08	11
-67	val_67	2008-04-08	11
-260	val_260	2008-04-08	11
-404	val_404	2008-04-08	11
-384	val_384	2008-04-08	11
-489	val_489	2008-04-08	11
-353	val_353	2008-04-08	11
-373	val_373	2008-04-08	11
-272	val_272	2008-04-08	11
-138	val_138	2008-04-08	11
-217	val_217	2008-04-08	11
-84	val_84	2008-04-08	11
-348	val_348	2008-04-08	11
-466	val_466	2008-04-08	11
-58	val_58	2008-04-08	11
-8	val_8	2008-04-08	11
-411	val_411	2008-04-08	11
-230	val_230	2008-04-08	11
-208	val_208	2008-04-08	11
-348	val_348	2008-04-08	11
-24	val_24	2008-04-08	11
-463	val_463	2008-04-08	11
-431	val_431	2008-04-08	11
-179	val_179	2008-04-08	11
-172	val_172	2008-04-08	11
-42	val_42	2008-04-08	11
-129	val_129	2008-04-08	11
-158	val_158	2008-04-08	11
-119	val_119	2008-04-08	11
-496	val_496	2008-04-08	11
-0	val_0	2008-04-08	11
-322	val_322	2008-04-08	11
-197	val_197	2008-04-08	11
-468	val_468	2008-04-08	11
-393	val_393	2008-04-08	11
-454	val_454	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-199	val_199	2008-04-08	11
-191	val_191	2008-04-08	11
-418	val_418	2008-04-08	11
-96	val_96	2008-04-08	11
-26	val_26	2008-04-08	11
-165	val_165	2008-04-08	11
-327	val_327	2008-04-08	11
-230	val_230	2008-04-08	11
-205	val_205	2008-04-08	11
-120	val_120	2008-04-08	11
-131	val_131	2008-04-08	11
-51	val_51	2008-04-08	11
-404	val_404	2008-04-08	11
-43	val_43	2008-04-08	11
-436	val_436	2008-04-08	11
-156	val_156	2008-04-08	11
-469	val_469	2008-04-08	11
-468	val_468	2008-04-08	11
-308	val_308	2008-04-08	11
-95	val_95	2008-04-08	11
-196	val_196	2008-04-08	11
-288	val_288	2008-04-08	11
-481	val_481	2008-04-08	11
-457	val_457	2008-04-08	11
-98	val_98	2008-04-08	11
-282	val_282	2008-04-08	11
-197	val_197	2008-04-08	11
-187	val_187	2008-04-08	11
-318	val_318	2008-04-08	11
-318	val_318	2008-04-08	11
-409	val_409	2008-04-08	11
-470	val_470	2008-04-08	11
-137	val_137	2008-04-08	11
-369	val_369	2008-04-08	11
-316	val_316	2008-04-08	11
-169	val_169	2008-04-08	11
-413	val_413	2008-04-08	11
-85	val_85	2008-04-08	11
-77	val_77	2008-04-08	11
-0	val_0	2008-04-08	11
-490	val_490	2008-04-08	11
-87	val_87	2008-04-08	11
-364	val_364	2008-04-08	11
-179	val_179	2008-04-08	11
-118	val_118	2008-04-08	11
-134	val_134	2008-04-08	11
-395	val_395	2008-04-08	11
-282	val_282	2008-04-08	11
-138	val_138	2008-04-08	11
-238	val_238	2008-04-08	11
-419	val_419	2008-04-08	11
-15	val_15	2008-04-08	11
-118	val_118	2008-04-08	11
-72	val_72	2008-04-08	11
-90	val_90	2008-04-08	11
-307	val_307	2008-04-08	11
-19	val_19	2008-04-08	11
-435	val_435	2008-04-08	11
-10	val_10	2008-04-08	11
-277	val_277	2008-04-08	11
-273	val_273	2008-04-08	11
-306	val_306	2008-04-08	11
-224	val_224	2008-04-08	11
-309	val_309	2008-04-08	11
-389	val_389	2008-04-08	11
-327	val_327	2008-04-08	11
-242	val_242	2008-04-08	11
-369	val_369	2008-04-08	11
-392	val_392	2008-04-08	11
-272	val_272	2008-04-08	11
-331	val_331	2008-04-08	11
-401	val_401	2008-04-08	11
-242	val_242	2008-04-08	11
-452	val_452	2008-04-08	11
-177	val_177	2008-04-08	11
-226	val_226	2008-04-08	11
-5	val_5	2008-04-08	11
-497	val_497	2008-04-08	11
-402	val_402	2008-04-08	11
-396	val_396	2008-04-08	11
-317	val_317	2008-04-08	11
-395	val_395	2008-04-08	11
-58	val_58	2008-04-08	11
-35	val_35	2008-04-08	11
-336	val_336	2008-04-08	11
-95	val_95	2008-04-08	11
-11	val_11	2008-04-08	11
-168	val_168	2008-04-08	11
-34	val_34	2008-04-08	11
-229	val_229	2008-04-08	11
-233	val_233	2008-04-08	11
-143	val_143	2008-04-08	11
-472	val_472	2008-04-08	11
-322	val_322	2008-04-08	11
-498	val_498	2008-04-08	11
-160	val_160	2008-04-08	11
-195	val_195	2008-04-08	11
-42	val_42	2008-04-08	11
-321	val_321	2008-04-08	11
-430	val_430	2008-04-08	11
-119	val_119	2008-04-08	11
-489	val_489	2008-04-08	11
-458	val_458	2008-04-08	11
-78	val_78	2008-04-08	11
-76	val_76	2008-04-08	11
-41	val_41	2008-04-08	11
-223	val_223	2008-04-08	11
-492	val_492	2008-04-08	11
-149	val_149	2008-04-08	11
-449	val_449	2008-04-08	11
-218	val_218	2008-04-08	11
-228	val_228	2008-04-08	11
-138	val_138	2008-04-08	11
-453	val_453	2008-04-08	11
-30	val_30	2008-04-08	11
-209	val_209	2008-04-08	11
-64	val_64	2008-04-08	11
-468	val_468	2008-04-08	11
-76	val_76	2008-04-08	11
-74	val_74	2008-04-08	11
-342	val_342	2008-04-08	11
-69	val_69	2008-04-08	11
-230	val_230	2008-04-08	11
-33	val_33	2008-04-08	11
-368	val_368	2008-04-08	11
-103	val_103	2008-04-08	11
-296	val_296	2008-04-08	11
-113	val_113	2008-04-08	11
-216	val_216	2008-04-08	11
-367	val_367	2008-04-08	11
-344	val_344	2008-04-08	11
-167	val_167	2008-04-08	11
-274	val_274	2008-04-08	11
-219	val_219	2008-04-08	11
-239	val_239	2008-04-08	11
-485	val_485	2008-04-08	11
-116	val_116	2008-04-08	11
-223	val_223	2008-04-08	11
-256	val_256	2008-04-08	11
-263	val_263	2008-04-08	11
-70	val_70	2008-04-08	11
-487	val_487	2008-04-08	11
-480	val_480	2008-04-08	11
-401	val_401	2008-04-08	11
-288	val_288	2008-04-08	11
-191	val_191	2008-04-08	11
-5	val_5	2008-04-08	11
-244	val_244	2008-04-08	11
-438	val_438	2008-04-08	11
-128	val_128	2008-04-08	11
-467	val_467	2008-04-08	11
-432	val_432	2008-04-08	11
-202	val_202	2008-04-08	11
-316	val_316	2008-04-08	11
-229	val_229	2008-04-08	11
-469	val_469	2008-04-08	11
-463	val_463	2008-04-08	11
-280	val_280	2008-04-08	11
-2	val_2	2008-04-08	11
-35	val_35	2008-04-08	11
-283	val_283	2008-04-08	11
-331	val_331	2008-04-08	11
-235	val_235	2008-04-08	11
-80	val_80	2008-04-08	11
-44	val_44	2008-04-08	11
-193	val_193	2008-04-08	11
-321	val_321	2008-04-08	11
-335	val_335	2008-04-08	11
-104	val_104	2008-04-08	11
-466	val_466	2008-04-08	11
-366	val_366	2008-04-08	11
-175	val_175	2008-04-08	11
-403	val_403	2008-04-08	11
-483	val_483	2008-04-08	11
-53	val_53	2008-04-08	11
-105	val_105	2008-04-08	11
-257	val_257	2008-04-08	11
-406	val_406	2008-04-08	11
-409	val_409	2008-04-08	11
-190	val_190	2008-04-08	11
-406	val_406	2008-04-08	11
-401	val_401	2008-04-08	11
-114	val_114	2008-04-08	11
-258	val_258	2008-04-08	11
-90	val_90	2008-04-08	11
-203	val_203	2008-04-08	11
-262	val_262	2008-04-08	11
-348	val_348	2008-04-08	11
-424	val_424	2008-04-08	11
-12	val_12	2008-04-08	11
-396	val_396	2008-04-08	11
-201	val_201	2008-04-08	11
-217	val_217	2008-04-08	11
-164	val_164	2008-04-08	11
-431	val_431	2008-04-08	11
-454	val_454	2008-04-08	11
-478	val_478	2008-04-08	11
-298	val_298	2008-04-08	11
-125	val_125	2008-04-08	11
-431	val_431	2008-04-08	11
-164	val_164	2008-04-08	11
-424	val_424	2008-04-08	11
-187	val_187	2008-04-08	11
-382	val_382	2008-04-08	11
-5	val_5	2008-04-08	11
-70	val_70	2008-04-08	11
-397	val_397	2008-04-08	11
-480	val_480	2008-04-08	11
-291	val_291	2008-04-08	11
-24	val_24	2008-04-08	11
-351	val_351	2008-04-08	11
-255	val_255	2008-04-08	11
-104	val_104	2008-04-08	11
-70	val_70	2008-04-08	11
-163	val_163	2008-04-08	11
-438	val_438	2008-04-08	11
-119	val_119	2008-04-08	11
-414	val_414	2008-04-08	11
-200	val_200	2008-04-08	11
-491	val_491	2008-04-08	11
-237	val_237	2008-04-08	11
-439	val_439	2008-04-08	11
-360	val_360	2008-04-08	11
-248	val_248	2008-04-08	11
-479	val_479	2008-04-08	11
-305	val_305	2008-04-08	11
-417	val_417	2008-04-08	11
-199	val_199	2008-04-08	11
-444	val_444	2008-04-08	11
-120	val_120	2008-04-08	11
-429	val_429	2008-04-08	11
-169	val_169	2008-04-08	11
-443	val_443	2008-04-08	11
-323	val_323	2008-04-08	11
-325	val_325	2008-04-08	11
-277	val_277	2008-04-08	11
-230	val_230	2008-04-08	11
-478	val_478	2008-04-08	11
-178	val_178	2008-04-08	11
-468	val_468	2008-04-08	11
-310	val_310	2008-04-08	11
-317	val_317	2008-04-08	11
-333	val_333	2008-04-08	11
-493	val_493	2008-04-08	11
-460	val_460	2008-04-08	11
-207	val_207	2008-04-08	11
-249	val_249	2008-04-08	11
-265	val_265	2008-04-08	11
-480	val_480	2008-04-08	11
-83	val_83	2008-04-08	11
-136	val_136	2008-04-08	11
-353	val_353	2008-04-08	11
-172	val_172	2008-04-08	11
-214	val_214	2008-04-08	11
-462	val_462	2008-04-08	11
-233	val_233	2008-04-08	11
-406	val_406	2008-04-08	11
-133	val_133	2008-04-08	11
-175	val_175	2008-04-08	11
-189	val_189	2008-04-08	11
-454	val_454	2008-04-08	11
-375	val_375	2008-04-08	11
-401	val_401	2008-04-08	11
-421	val_421	2008-04-08	11
-407	val_407	2008-04-08	11
-384	val_384	2008-04-08	11
-256	val_256	2008-04-08	11
-26	val_26	2008-04-08	11
-134	val_134	2008-04-08	11
-67	val_67	2008-04-08	11
-384	val_384	2008-04-08	11
-379	val_379	2008-04-08	11
-18	val_18	2008-04-08	11
-462	val_462	2008-04-08	11
-492	val_492	2008-04-08	11
-100	val_100	2008-04-08	11
-298	val_298	2008-04-08	11
-9	val_9	2008-04-08	11
-341	val_341	2008-04-08	11
-498	val_498	2008-04-08	11
-146	val_146	2008-04-08	11
-458	val_458	2008-04-08	11
-362	val_362	2008-04-08	11
-186	val_186	2008-04-08	11
-285	val_285	2008-04-08	11
-348	val_348	2008-04-08	11
-167	val_167	2008-04-08	11
-18	val_18	2008-04-08	11
-273	val_273	2008-04-08	11
-183	val_183	2008-04-08	11
-281	val_281	2008-04-08	11
-344	val_344	2008-04-08	11
-97	val_97	2008-04-08	11
-469	val_469	2008-04-08	11
-315	val_315	2008-04-08	11
-84	val_84	2008-04-08	11
-28	val_28	2008-04-08	11
-37	val_37	2008-04-08	11
-448	val_448	2008-04-08	11
-152	val_152	2008-04-08	11
-348	val_348	2008-04-08	11
-307	val_307	2008-04-08	11
-194	val_194	2008-04-08	11
-414	val_414	2008-04-08	11
-477	val_477	2008-04-08	11
-222	val_222	2008-04-08	11
-126	val_126	2008-04-08	11
-90	val_90	2008-04-08	11
-169	val_169	2008-04-08	11
-403	val_403	2008-04-08	11
-400	val_400	2008-04-08	11
-200	val_200	2008-04-08	11
-97	val_97	2008-04-08	11
-238	val_238	2008-04-08	12
-86	val_86	2008-04-08	12
-311	val_311	2008-04-08	12
-27	val_27	2008-04-08	12
-165	val_165	2008-04-08	12
-409	val_409	2008-04-08	12
-255	val_255	2008-04-08	12
-278	val_278	2008-04-08	12
-98	val_98	2008-04-08	12
-484	val_484	2008-04-08	12
-265	val_265	2008-04-08	12
-193	val_193	2008-04-08	12
-401	val_401	2008-04-08	12
-150	val_150	2008-04-08	12
-273	val_273	2008-04-08	12
-224	val_224	2008-04-08	12
-369	val_369	2008-04-08	12
-66	val_66	2008-04-08	12
-128	val_128	2008-04-08	12
-213	val_213	2008-04-08	12
-146	val_146	2008-04-08	12
-406	val_406	2008-04-08	12
-429	val_429	2008-04-08	12
-374	val_374	2008-04-08	12
-152	val_152	2008-04-08	12
-469	val_469	2008-04-08	12
-145	val_145	2008-04-08	12
-495	val_495	2008-04-08	12
-37	val_37	2008-04-08	12
-327	val_327	2008-04-08	12
-281	val_281	2008-04-08	12
-277	val_277	2008-04-08	12
-209	val_209	2008-04-08	12
-15	val_15	2008-04-08	12
-82	val_82	2008-04-08	12
-403	val_403	2008-04-08	12
-166	val_166	2008-04-08	12
-417	val_417	2008-04-08	12
-430	val_430	2008-04-08	12
-252	val_252	2008-04-08	12
-292	val_292	2008-04-08	12
-219	val_219	2008-04-08	12
-287	val_287	2008-04-08	12
-153	val_153	2008-04-08	12
-193	val_193	2008-04-08	12
-338	val_338	2008-04-08	12
-446	val_446	2008-04-08	12
-459	val_459	2008-04-08	12
-394	val_394	2008-04-08	12
-237	val_237	2008-04-08	12
-482	val_482	2008-04-08	12
-174	val_174	2008-04-08	12
-413	val_413	2008-04-08	12
-494	val_494	2008-04-08	12
-207	val_207	2008-04-08	12
-199	val_199	2008-04-08	12
-466	val_466	2008-04-08	12
-208	val_208	2008-04-08	12
-174	val_174	2008-04-08	12
-399	val_399	2008-04-08	12
-396	val_396	2008-04-08	12
-247	val_247	2008-04-08	12
-417	val_417	2008-04-08	12
-489	val_489	2008-04-08	12
-162	val_162	2008-04-08	12
-377	val_377	2008-04-08	12
-397	val_397	2008-04-08	12
-309	val_309	2008-04-08	12
-365	val_365	2008-04-08	12
-266	val_266	2008-04-08	12
-439	val_439	2008-04-08	12
-342	val_342	2008-04-08	12
-367	val_367	2008-04-08	12
-325	val_325	2008-04-08	12
-167	val_167	2008-04-08	12
-195	val_195	2008-04-08	12
-475	val_475	2008-04-08	12
-17	val_17	2008-04-08	12
-113	val_113	2008-04-08	12
-155	val_155	2008-04-08	12
-203	val_203	2008-04-08	12
-339	val_339	2008-04-08	12
-0	val_0	2008-04-08	12
-455	val_455	2008-04-08	12
-128	val_128	2008-04-08	12
-311	val_311	2008-04-08	12
-316	val_316	2008-04-08	12
-57	val_57	2008-04-08	12
-302	val_302	2008-04-08	12
-205	val_205	2008-04-08	12
-149	val_149	2008-04-08	12
-438	val_438	2008-04-08	12
-345	val_345	2008-04-08	12
-129	val_129	2008-04-08	12
-170	val_170	2008-04-08	12
-20	val_20	2008-04-08	12
-489	val_489	2008-04-08	12
-157	val_157	2008-04-08	12
-378	val_378	2008-04-08	12
-221	val_221	2008-04-08	12
-92	val_92	2008-04-08	12
-111	val_111	2008-04-08	12
-47	val_47	2008-04-08	12
-72	val_72	2008-04-08	12
-4	val_4	2008-04-08	12
-280	val_280	2008-04-08	12
-35	val_35	2008-04-08	12
-427	val_427	2008-04-08	12
-277	val_277	2008-04-08	12
-208	val_208	2008-04-08	12
-356	val_356	2008-04-08	12
-399	val_399	2008-04-08	12
-169	val_169	2008-04-08	12
-382	val_382	2008-04-08	12
-498	val_498	2008-04-08	12
-125	val_125	2008-04-08	12
-386	val_386	2008-04-08	12
-437	val_437	2008-04-08	12
-469	val_469	2008-04-08	12
-192	val_192	2008-04-08	12
-286	val_286	2008-04-08	12
-187	val_187	2008-04-08	12
-176	val_176	2008-04-08	12
-54	val_54	2008-04-08	12
-459	val_459	2008-04-08	12
-51	val_51	2008-04-08	12
-138	val_138	2008-04-08	12
-103	val_103	2008-04-08	12
-239	val_239	2008-04-08	12
-213	val_213	2008-04-08	12
-216	val_216	2008-04-08	12
-430	val_430	2008-04-08	12
-278	val_278	2008-04-08	12
-176	val_176	2008-04-08	12
-289	val_289	2008-04-08	12
-221	val_221	2008-04-08	12
-65	val_65	2008-04-08	12
-318	val_318	2008-04-08	12
-332	val_332	2008-04-08	12
-311	val_311	2008-04-08	12
-275	val_275	2008-04-08	12
-137	val_137	2008-04-08	12
-241	val_241	2008-04-08	12
-83	val_83	2008-04-08	12
-333	val_333	2008-04-08	12
-180	val_180	2008-04-08	12
-284	val_284	2008-04-08	12
-12	val_12	2008-04-08	12
-230	val_230	2008-04-08	12
-181	val_181	2008-04-08	12
-67	val_67	2008-04-08	12
-260	val_260	2008-04-08	12
-404	val_404	2008-04-08	12
-384	val_384	2008-04-08	12
-489	val_489	2008-04-08	12
-353	val_353	2008-04-08	12
-373	val_373	2008-04-08	12
-272	val_272	2008-04-08	12
-138	val_138	2008-04-08	12
-217	val_217	2008-04-08	12
-84	val_84	2008-04-08	12
-348	val_348	2008-04-08	12
-466	val_466	2008-04-08	12
-58	val_58	2008-04-08	12
-8	val_8	2008-04-08	12
-411	val_411	2008-04-08	12
-230	val_230	2008-04-08	12
-208	val_208	2008-04-08	12
-348	val_348	2008-04-08	12
-24	val_24	2008-04-08	12
-463	val_463	2008-04-08	12
-431	val_431	2008-04-08	12
-179	val_179	2008-04-08	12
-172	val_172	2008-04-08	12
-42	val_42	2008-04-08	12
-129	val_129	2008-04-08	12
-158	val_158	2008-04-08	12
-119	val_119	2008-04-08	12
-496	val_496	2008-04-08	12
-0	val_0	2008-04-08	12
-322	val_322	2008-04-08	12
-197	val_197	2008-04-08	12
-468	val_468	2008-04-08	12
-393	val_393	2008-04-08	12
-454	val_454	2008-04-08	12
-100	val_100	2008-04-08	12
-298	val_298	2008-04-08	12
-199	val_199	2008-04-08	12
-191	val_191	2008-04-08	12
-418	val_418	2008-04-08	12
-96	val_96	2008-04-08	12
-26	val_26	2008-04-08	12
-165	val_165	2008-04-08	12
-327	val_327	2008-04-08	12
-230	val_230	2008-04-08	12
-205	val_205	2008-04-08	12
-120	val_120	2008-04-08	12
-131	val_131	2008-04-08	12
-51	val_51	2008-04-08	12
-404	val_404	2008-04-08	12
-43	val_43	2008-04-08	12
-436	val_436	2008-04-08	12
-156	val_156	2008-04-08	12
-469	val_469	2008-04-08	12
-468	val_468	2008-04-08	12
-308	val_308	2008-04-08	12
-95	val_95	2008-04-08	12
-196	val_196	2008-04-08	12
-288	val_288	2008-04-08	12
-481	val_481	2008-04-08	12
-457	val_457	2008-04-08	12
-98	val_98	2008-04-08	12
-282	val_282	2008-04-08	12
-197	val_197	2008-04-08	12
-187	val_187	2008-04-08	12
-318	val_318	2008-04-08	12
-318	val_318	2008-04-08	12
-409	val_409	2008-04-08	12
-470	val_470	2008-04-08	12
-137	val_137	2008-04-08	12
-369	val_369	2008-04-08	12
-316	val_316	2008-04-08	12
-169	val_169	2008-04-08	12
-413	val_413	2008-04-08	12
-85	val_85	2008-04-08	12
-77	val_77	2008-04-08	12
-0	val_0	2008-04-08	12
-490	val_490	2008-04-08	12
-87	val_87	2008-04-08	12
-364	val_364	2008-04-08	12
-179	val_179	2008-04-08	12
-118	val_118	2008-04-08	12
-134	val_134	2008-04-08	12
-395	val_395	2008-04-08	12
-282	val_282	2008-04-08	12
-138	val_138	2008-04-08	12
-238	val_238	2008-04-08	12
-419	val_419	2008-04-08	12
-15	val_15	2008-04-08	12
-118	val_118	2008-04-08	12
-72	val_72	2008-04-08	12
-90	val_90	2008-04-08	12
-307	val_307	2008-04-08	12
-19	val_19	2008-04-08	12
-435	val_435	2008-04-08	12
-10	val_10	2008-04-08	12
-277	val_277	2008-04-08	12
-273	val_273	2008-04-08	12
-306	val_306	2008-04-08	12
-224	val_224	2008-04-08	12
-309	val_309	2008-04-08	12
-389	val_389	2008-04-08	12
-327	val_327	2008-04-08	12
-242	val_242	2008-04-08	12
-369	val_369	2008-04-08	12
-392	val_392	2008-04-08	12
-272	val_272	2008-04-08	12
-331	val_331	2008-04-08	12
-401	val_401	2008-04-08	12
-242	val_242	2008-04-08	12
-452	val_452	2008-04-08	12
-177	val_177	2008-04-08	12
-226	val_226	2008-04-08	12
-5	val_5	2008-04-08	12
-497	val_497	2008-04-08	12
-402	val_402	2008-04-08	12
-396	val_396	2008-04-08	12
-317	val_317	2008-04-08	12
-395	val_395	2008-04-08	12
-58	val_58	2008-04-08	12
-35	val_35	2008-04-08	12
-336	val_336	2008-04-08	12
-95	val_95	2008-04-08	12
-11	val_11	2008-04-08	12
-168	val_168	2008-04-08	12
-34	val_34	2008-04-08	12
-229	val_229	2008-04-08	12
-233	val_233	2008-04-08	12
-143	val_143	2008-04-08	12
-472	val_472	2008-04-08	12
-322	val_322	2008-04-08	12
-498	val_498	2008-04-08	12
-160	val_160	2008-04-08	12
-195	val_195	2008-04-08	12
-42	val_42	2008-04-08	12
-321	val_321	2008-04-08	12
-430	val_430	2008-04-08	12
-119	val_119	2008-04-08	12
-489	val_489	2008-04-08	12
-458	val_458	2008-04-08	12
-78	val_78	2008-04-08	12
-76	val_76	2008-04-08	12
-41	val_41	2008-04-08	12
-223	val_223	2008-04-08	12
-492	val_492	2008-04-08	12
-149	val_149	2008-04-08	12
-449	val_449	2008-04-08	12
-218	val_218	2008-04-08	12
-228	val_228	2008-04-08	12
-138	val_138	2008-04-08	12
-453	val_453	2008-04-08	12
-30	val_30	2008-04-08	12
-209	val_209	2008-04-08	12
-64	val_64	2008-04-08	12
-468	val_468	2008-04-08	12
-76	val_76	2008-04-08	12
-74	val_74	2008-04-08	12
-342	val_342	2008-04-08	12
-69	val_69	2008-04-08	12
-230	val_230	2008-04-08	12
-33	val_33	2008-04-08	12
-368	val_368	2008-04-08	12
-103	val_103	2008-04-08	12
-296	val_296	2008-04-08	12
-113	val_113	2008-04-08	12
-216	val_216	2008-04-08	12
-367	val_367	2008-04-08	12
-344	val_344	2008-04-08	12
-167	val_167	2008-04-08	12
-274	val_274	2008-04-08	12
-219	val_219	2008-04-08	12
-239	val_239	2008-04-08	12
-485	val_485	2008-04-08	12
-116	val_116	2008-04-08	12
-223	val_223	2008-04-08	12
-256	val_256	2008-04-08	12
-263	val_263	2008-04-08	12
-70	val_70	2008-04-08	12
-487	val_487	2008-04-08	12
-480	val_480	2008-04-08	12
-401	val_401	2008-04-08	12
-288	val_288	2008-04-08	12
-191	val_191	2008-04-08	12
-5	val_5	2008-04-08	12
-244	val_244	2008-04-08	12
-438	val_438	2008-04-08	12
-128	val_128	2008-04-08	12
-467	val_467	2008-04-08	12
-432	val_432	2008-04-08	12
-202	val_202	2008-04-08	12
-316	val_316	2008-04-08	12
-229	val_229	2008-04-08	12
-469	val_469	2008-04-08	12
-463	val_463	2008-04-08	12
-280	val_280	2008-04-08	12
-2	val_2	2008-04-08	12
-35	val_35	2008-04-08	12
-283	val_283	2008-04-08	12
-331	val_331	2008-04-08	12
-235	val_235	2008-04-08	12
-80	val_80	2008-04-08	12
-44	val_44	2008-04-08	12
-193	val_193	2008-04-08	12
-321	val_321	2008-04-08	12
-335	val_335	2008-04-08	12
-104	val_104	2008-04-08	12
-466	val_466	2008-04-08	12
-366	val_366	2008-04-08	12
-175	val_175	2008-04-08	12
-403	val_403	2008-04-08	12
-483	val_483	2008-04-08	12
-53	val_53	2008-04-08	12
-105	val_105	2008-04-08	12
-257	val_257	2008-04-08	12
-406	val_406	2008-04-08	12
-409	val_409	2008-04-08	12
-190	val_190	2008-04-08	12
-406	val_406	2008-04-08	12
-401	val_401	2008-04-08	12
-114	val_114	2008-04-08	12
-258	val_258	2008-04-08	12
-90	val_90	2008-04-08	12
-203	val_203	2008-04-08	12
-262	val_262	2008-04-08	12
-348	val_348	2008-04-08	12
-424	val_424	2008-04-08	12
-12	val_12	2008-04-08	12
-396	val_396	2008-04-08	12
-201	val_201	2008-04-08	12
-217	val_217	2008-04-08	12
-164	val_164	2008-04-08	12
-431	val_431	2008-04-08	12
-454	val_454	2008-04-08	12
-478	val_478	2008-04-08	12
-298	val_298	2008-04-08	12
-125	val_125	2008-04-08	12
-431	val_431	2008-04-08	12
-164	val_164	2008-04-08	12
-424	val_424	2008-04-08	12
-187	val_187	2008-04-08	12
-382	val_382	2008-04-08	12
-5	val_5	2008-04-08	12
-70	val_70	2008-04-08	12
-397	val_397	2008-04-08	12
-480	val_480	2008-04-08	12
-291	val_291	2008-04-08	12
-24	val_24	2008-04-08	12
-351	val_351	2008-04-08	12
-255	val_255	2008-04-08	12
-104	val_104	2008-04-08	12
-70	val_70	2008-04-08	12
-163	val_163	2008-04-08	12
-438	val_438	2008-04-08	12
-119	val_119	2008-04-08	12
-414	val_414	2008-04-08	12
-200	val_200	2008-04-08	12
-491	val_491	2008-04-08	12
-237	val_237	2008-04-08	12
-439	val_439	2008-04-08	12
-360	val_360	2008-04-08	12
-248	val_248	2008-04-08	12
-479	val_479	2008-04-08	12
-305	val_305	2008-04-08	12
-417	val_417	2008-04-08	12
-199	val_199	2008-04-08	12
-444	val_444	2008-04-08	12
-120	val_120	2008-04-08	12
-429	val_429	2008-04-08	12
-169	val_169	2008-04-08	12
-443	val_443	2008-04-08	12
-323	val_323	2008-04-08	12
-325	val_325	2008-04-08	12
-277	val_277	2008-04-08	12
-230	val_230	2008-04-08	12
-478	val_478	2008-04-08	12
-178	val_178	2008-04-08	12
-468	val_468	2008-04-08	12
-310	val_310	2008-04-08	12
-317	val_317	2008-04-08	12
-333	val_333	2008-04-08	12
-493	val_493	2008-04-08	12
-460	val_460	2008-04-08	12
-207	val_207	2008-04-08	12
-249	val_249	2008-04-08	12
-265	val_265	2008-04-08	12
-480	val_480	2008-04-08	12
-83	val_83	2008-04-08	12
-136	val_136	2008-04-08	12
-353	val_353	2008-04-08	12
-172	val_172	2008-04-08	12
-214	val_214	2008-04-08	12
-462	val_462	2008-04-08	12
-233	val_233	2008-04-08	12
-406	val_406	2008-04-08	12
-133	val_133	2008-04-08	12
-175	val_175	2008-04-08	12
-189	val_189	2008-04-08	12
-454	val_454	2008-04-08	12
-375	val_375	2008-04-08	12
-401	val_401	2008-04-08	12
-421	val_421	2008-04-08	12
-407	val_407	2008-04-08	12
-384	val_384	2008-04-08	12
-256	val_256	2008-04-08	12
-26	val_26	2008-04-08	12
-134	val_134	2008-04-08	12
-67	val_67	2008-04-08	12
-384	val_384	2008-04-08	12
-379	val_379	2008-04-08	12
-18	val_18	2008-04-08	12
-462	val_462	2008-04-08	12
-492	val_492	2008-04-08	12
-100	val_100	2008-04-08	12
-298	val_298	2008-04-08	12
-9	val_9	2008-04-08	12
-341	val_341	2008-04-08	12
-498	val_498	2008-04-08	12
-146	val_146	2008-04-08	12
-458	val_458	2008-04-08	12
-362	val_362	2008-04-08	12
-186	val_186	2008-04-08	12
-285	val_285	2008-04-08	12
-348	val_348	2008-04-08	12
-167	val_167	2008-04-08	12
-18	val_18	2008-04-08	12
-273	val_273	2008-04-08	12
-183	val_183	2008-04-08	12
-281	val_281	2008-04-08	12
-344	val_344	2008-04-08	12
-97	val_97	2008-04-08	12
-469	val_469	2008-04-08	12
-315	val_315	2008-04-08	12
-84	val_84	2008-04-08	12
-28	val_28	2008-04-08	12
-37	val_37	2008-04-08	12
-448	val_448	2008-04-08	12
-152	val_152	2008-04-08	12
-348	val_348	2008-04-08	12
-307	val_307	2008-04-08	12
-194	val_194	2008-04-08	12
-414	val_414	2008-04-08	12
-477	val_477	2008-04-08	12
-222	val_222	2008-04-08	12
-126	val_126	2008-04-08	12
-90	val_90	2008-04-08	12
-169	val_169	2008-04-08	12
-403	val_403	2008-04-08	12
-400	val_400	2008-04-08	12
-200	val_200	2008-04-08	12
-97	val_97	2008-04-08	12
diff --git a/sql/hive/src/test/resources/golden/stats4-14-9c82167763a771c175c656786d545798 b/sql/hive/src/test/resources/golden/stats4-14-9c82167763a771c175c656786d545798
deleted file mode 100644
index 653516475da222e533bd2651cda6fdb1cdc1576a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats4-14-9c82167763a771c175c656786d545798
+++ /dev/null
@@ -1,1000 +0,0 @@
-238	val_238	2008-12-31	11
-86	val_86	2008-12-31	11
-311	val_311	2008-12-31	11
-27	val_27	2008-12-31	11
-165	val_165	2008-12-31	11
-409	val_409	2008-12-31	11
-255	val_255	2008-12-31	11
-278	val_278	2008-12-31	11
-98	val_98	2008-12-31	11
-484	val_484	2008-12-31	11
-265	val_265	2008-12-31	11
-193	val_193	2008-12-31	11
-401	val_401	2008-12-31	11
-150	val_150	2008-12-31	11
-273	val_273	2008-12-31	11
-224	val_224	2008-12-31	11
-369	val_369	2008-12-31	11
-66	val_66	2008-12-31	11
-128	val_128	2008-12-31	11
-213	val_213	2008-12-31	11
-146	val_146	2008-12-31	11
-406	val_406	2008-12-31	11
-429	val_429	2008-12-31	11
-374	val_374	2008-12-31	11
-152	val_152	2008-12-31	11
-469	val_469	2008-12-31	11
-145	val_145	2008-12-31	11
-495	val_495	2008-12-31	11
-37	val_37	2008-12-31	11
-327	val_327	2008-12-31	11
-281	val_281	2008-12-31	11
-277	val_277	2008-12-31	11
-209	val_209	2008-12-31	11
-15	val_15	2008-12-31	11
-82	val_82	2008-12-31	11
-403	val_403	2008-12-31	11
-166	val_166	2008-12-31	11
-417	val_417	2008-12-31	11
-430	val_430	2008-12-31	11
-252	val_252	2008-12-31	11
-292	val_292	2008-12-31	11
-219	val_219	2008-12-31	11
-287	val_287	2008-12-31	11
-153	val_153	2008-12-31	11
-193	val_193	2008-12-31	11
-338	val_338	2008-12-31	11
-446	val_446	2008-12-31	11
-459	val_459	2008-12-31	11
-394	val_394	2008-12-31	11
-237	val_237	2008-12-31	11
-482	val_482	2008-12-31	11
-174	val_174	2008-12-31	11
-413	val_413	2008-12-31	11
-494	val_494	2008-12-31	11
-207	val_207	2008-12-31	11
-199	val_199	2008-12-31	11
-466	val_466	2008-12-31	11
-208	val_208	2008-12-31	11
-174	val_174	2008-12-31	11
-399	val_399	2008-12-31	11
-396	val_396	2008-12-31	11
-247	val_247	2008-12-31	11
-417	val_417	2008-12-31	11
-489	val_489	2008-12-31	11
-162	val_162	2008-12-31	11
-377	val_377	2008-12-31	11
-397	val_397	2008-12-31	11
-309	val_309	2008-12-31	11
-365	val_365	2008-12-31	11
-266	val_266	2008-12-31	11
-439	val_439	2008-12-31	11
-342	val_342	2008-12-31	11
-367	val_367	2008-12-31	11
-325	val_325	2008-12-31	11
-167	val_167	2008-12-31	11
-195	val_195	2008-12-31	11
-475	val_475	2008-12-31	11
-17	val_17	2008-12-31	11
-113	val_113	2008-12-31	11
-155	val_155	2008-12-31	11
-203	val_203	2008-12-31	11
-339	val_339	2008-12-31	11
-0	val_0	2008-12-31	11
-455	val_455	2008-12-31	11
-128	val_128	2008-12-31	11
-311	val_311	2008-12-31	11
-316	val_316	2008-12-31	11
-57	val_57	2008-12-31	11
-302	val_302	2008-12-31	11
-205	val_205	2008-12-31	11
-149	val_149	2008-12-31	11
-438	val_438	2008-12-31	11
-345	val_345	2008-12-31	11
-129	val_129	2008-12-31	11
-170	val_170	2008-12-31	11
-20	val_20	2008-12-31	11
-489	val_489	2008-12-31	11
-157	val_157	2008-12-31	11
-378	val_378	2008-12-31	11
-221	val_221	2008-12-31	11
-92	val_92	2008-12-31	11
-111	val_111	2008-12-31	11
-47	val_47	2008-12-31	11
-72	val_72	2008-12-31	11
-4	val_4	2008-12-31	11
-280	val_280	2008-12-31	11
-35	val_35	2008-12-31	11
-427	val_427	2008-12-31	11
-277	val_277	2008-12-31	11
-208	val_208	2008-12-31	11
-356	val_356	2008-12-31	11
-399	val_399	2008-12-31	11
-169	val_169	2008-12-31	11
-382	val_382	2008-12-31	11
-498	val_498	2008-12-31	11
-125	val_125	2008-12-31	11
-386	val_386	2008-12-31	11
-437	val_437	2008-12-31	11
-469	val_469	2008-12-31	11
-192	val_192	2008-12-31	11
-286	val_286	2008-12-31	11
-187	val_187	2008-12-31	11
-176	val_176	2008-12-31	11
-54	val_54	2008-12-31	11
-459	val_459	2008-12-31	11
-51	val_51	2008-12-31	11
-138	val_138	2008-12-31	11
-103	val_103	2008-12-31	11
-239	val_239	2008-12-31	11
-213	val_213	2008-12-31	11
-216	val_216	2008-12-31	11
-430	val_430	2008-12-31	11
-278	val_278	2008-12-31	11
-176	val_176	2008-12-31	11
-289	val_289	2008-12-31	11
-221	val_221	2008-12-31	11
-65	val_65	2008-12-31	11
-318	val_318	2008-12-31	11
-332	val_332	2008-12-31	11
-311	val_311	2008-12-31	11
-275	val_275	2008-12-31	11
-137	val_137	2008-12-31	11
-241	val_241	2008-12-31	11
-83	val_83	2008-12-31	11
-333	val_333	2008-12-31	11
-180	val_180	2008-12-31	11
-284	val_284	2008-12-31	11
-12	val_12	2008-12-31	11
-230	val_230	2008-12-31	11
-181	val_181	2008-12-31	11
-67	val_67	2008-12-31	11
-260	val_260	2008-12-31	11
-404	val_404	2008-12-31	11
-384	val_384	2008-12-31	11
-489	val_489	2008-12-31	11
-353	val_353	2008-12-31	11
-373	val_373	2008-12-31	11
-272	val_272	2008-12-31	11
-138	val_138	2008-12-31	11
-217	val_217	2008-12-31	11
-84	val_84	2008-12-31	11
-348	val_348	2008-12-31	11
-466	val_466	2008-12-31	11
-58	val_58	2008-12-31	11
-8	val_8	2008-12-31	11
-411	val_411	2008-12-31	11
-230	val_230	2008-12-31	11
-208	val_208	2008-12-31	11
-348	val_348	2008-12-31	11
-24	val_24	2008-12-31	11
-463	val_463	2008-12-31	11
-431	val_431	2008-12-31	11
-179	val_179	2008-12-31	11
-172	val_172	2008-12-31	11
-42	val_42	2008-12-31	11
-129	val_129	2008-12-31	11
-158	val_158	2008-12-31	11
-119	val_119	2008-12-31	11
-496	val_496	2008-12-31	11
-0	val_0	2008-12-31	11
-322	val_322	2008-12-31	11
-197	val_197	2008-12-31	11
-468	val_468	2008-12-31	11
-393	val_393	2008-12-31	11
-454	val_454	2008-12-31	11
-100	val_100	2008-12-31	11
-298	val_298	2008-12-31	11
-199	val_199	2008-12-31	11
-191	val_191	2008-12-31	11
-418	val_418	2008-12-31	11
-96	val_96	2008-12-31	11
-26	val_26	2008-12-31	11
-165	val_165	2008-12-31	11
-327	val_327	2008-12-31	11
-230	val_230	2008-12-31	11
-205	val_205	2008-12-31	11
-120	val_120	2008-12-31	11
-131	val_131	2008-12-31	11
-51	val_51	2008-12-31	11
-404	val_404	2008-12-31	11
-43	val_43	2008-12-31	11
-436	val_436	2008-12-31	11
-156	val_156	2008-12-31	11
-469	val_469	2008-12-31	11
-468	val_468	2008-12-31	11
-308	val_308	2008-12-31	11
-95	val_95	2008-12-31	11
-196	val_196	2008-12-31	11
-288	val_288	2008-12-31	11
-481	val_481	2008-12-31	11
-457	val_457	2008-12-31	11
-98	val_98	2008-12-31	11
-282	val_282	2008-12-31	11
-197	val_197	2008-12-31	11
-187	val_187	2008-12-31	11
-318	val_318	2008-12-31	11
-318	val_318	2008-12-31	11
-409	val_409	2008-12-31	11
-470	val_470	2008-12-31	11
-137	val_137	2008-12-31	11
-369	val_369	2008-12-31	11
-316	val_316	2008-12-31	11
-169	val_169	2008-12-31	11
-413	val_413	2008-12-31	11
-85	val_85	2008-12-31	11
-77	val_77	2008-12-31	11
-0	val_0	2008-12-31	11
-490	val_490	2008-12-31	11
-87	val_87	2008-12-31	11
-364	val_364	2008-12-31	11
-179	val_179	2008-12-31	11
-118	val_118	2008-12-31	11
-134	val_134	2008-12-31	11
-395	val_395	2008-12-31	11
-282	val_282	2008-12-31	11
-138	val_138	2008-12-31	11
-238	val_238	2008-12-31	11
-419	val_419	2008-12-31	11
-15	val_15	2008-12-31	11
-118	val_118	2008-12-31	11
-72	val_72	2008-12-31	11
-90	val_90	2008-12-31	11
-307	val_307	2008-12-31	11
-19	val_19	2008-12-31	11
-435	val_435	2008-12-31	11
-10	val_10	2008-12-31	11
-277	val_277	2008-12-31	11
-273	val_273	2008-12-31	11
-306	val_306	2008-12-31	11
-224	val_224	2008-12-31	11
-309	val_309	2008-12-31	11
-389	val_389	2008-12-31	11
-327	val_327	2008-12-31	11
-242	val_242	2008-12-31	11
-369	val_369	2008-12-31	11
-392	val_392	2008-12-31	11
-272	val_272	2008-12-31	11
-331	val_331	2008-12-31	11
-401	val_401	2008-12-31	11
-242	val_242	2008-12-31	11
-452	val_452	2008-12-31	11
-177	val_177	2008-12-31	11
-226	val_226	2008-12-31	11
-5	val_5	2008-12-31	11
-497	val_497	2008-12-31	11
-402	val_402	2008-12-31	11
-396	val_396	2008-12-31	11
-317	val_317	2008-12-31	11
-395	val_395	2008-12-31	11
-58	val_58	2008-12-31	11
-35	val_35	2008-12-31	11
-336	val_336	2008-12-31	11
-95	val_95	2008-12-31	11
-11	val_11	2008-12-31	11
-168	val_168	2008-12-31	11
-34	val_34	2008-12-31	11
-229	val_229	2008-12-31	11
-233	val_233	2008-12-31	11
-143	val_143	2008-12-31	11
-472	val_472	2008-12-31	11
-322	val_322	2008-12-31	11
-498	val_498	2008-12-31	11
-160	val_160	2008-12-31	11
-195	val_195	2008-12-31	11
-42	val_42	2008-12-31	11
-321	val_321	2008-12-31	11
-430	val_430	2008-12-31	11
-119	val_119	2008-12-31	11
-489	val_489	2008-12-31	11
-458	val_458	2008-12-31	11
-78	val_78	2008-12-31	11
-76	val_76	2008-12-31	11
-41	val_41	2008-12-31	11
-223	val_223	2008-12-31	11
-492	val_492	2008-12-31	11
-149	val_149	2008-12-31	11
-449	val_449	2008-12-31	11
-218	val_218	2008-12-31	11
-228	val_228	2008-12-31	11
-138	val_138	2008-12-31	11
-453	val_453	2008-12-31	11
-30	val_30	2008-12-31	11
-209	val_209	2008-12-31	11
-64	val_64	2008-12-31	11
-468	val_468	2008-12-31	11
-76	val_76	2008-12-31	11
-74	val_74	2008-12-31	11
-342	val_342	2008-12-31	11
-69	val_69	2008-12-31	11
-230	val_230	2008-12-31	11
-33	val_33	2008-12-31	11
-368	val_368	2008-12-31	11
-103	val_103	2008-12-31	11
-296	val_296	2008-12-31	11
-113	val_113	2008-12-31	11
-216	val_216	2008-12-31	11
-367	val_367	2008-12-31	11
-344	val_344	2008-12-31	11
-167	val_167	2008-12-31	11
-274	val_274	2008-12-31	11
-219	val_219	2008-12-31	11
-239	val_239	2008-12-31	11
-485	val_485	2008-12-31	11
-116	val_116	2008-12-31	11
-223	val_223	2008-12-31	11
-256	val_256	2008-12-31	11
-263	val_263	2008-12-31	11
-70	val_70	2008-12-31	11
-487	val_487	2008-12-31	11
-480	val_480	2008-12-31	11
-401	val_401	2008-12-31	11
-288	val_288	2008-12-31	11
-191	val_191	2008-12-31	11
-5	val_5	2008-12-31	11
-244	val_244	2008-12-31	11
-438	val_438	2008-12-31	11
-128	val_128	2008-12-31	11
-467	val_467	2008-12-31	11
-432	val_432	2008-12-31	11
-202	val_202	2008-12-31	11
-316	val_316	2008-12-31	11
-229	val_229	2008-12-31	11
-469	val_469	2008-12-31	11
-463	val_463	2008-12-31	11
-280	val_280	2008-12-31	11
-2	val_2	2008-12-31	11
-35	val_35	2008-12-31	11
-283	val_283	2008-12-31	11
-331	val_331	2008-12-31	11
-235	val_235	2008-12-31	11
-80	val_80	2008-12-31	11
-44	val_44	2008-12-31	11
-193	val_193	2008-12-31	11
-321	val_321	2008-12-31	11
-335	val_335	2008-12-31	11
-104	val_104	2008-12-31	11
-466	val_466	2008-12-31	11
-366	val_366	2008-12-31	11
-175	val_175	2008-12-31	11
-403	val_403	2008-12-31	11
-483	val_483	2008-12-31	11
-53	val_53	2008-12-31	11
-105	val_105	2008-12-31	11
-257	val_257	2008-12-31	11
-406	val_406	2008-12-31	11
-409	val_409	2008-12-31	11
-190	val_190	2008-12-31	11
-406	val_406	2008-12-31	11
-401	val_401	2008-12-31	11
-114	val_114	2008-12-31	11
-258	val_258	2008-12-31	11
-90	val_90	2008-12-31	11
-203	val_203	2008-12-31	11
-262	val_262	2008-12-31	11
-348	val_348	2008-12-31	11
-424	val_424	2008-12-31	11
-12	val_12	2008-12-31	11
-396	val_396	2008-12-31	11
-201	val_201	2008-12-31	11
-217	val_217	2008-12-31	11
-164	val_164	2008-12-31	11
-431	val_431	2008-12-31	11
-454	val_454	2008-12-31	11
-478	val_478	2008-12-31	11
-298	val_298	2008-12-31	11
-125	val_125	2008-12-31	11
-431	val_431	2008-12-31	11
-164	val_164	2008-12-31	11
-424	val_424	2008-12-31	11
-187	val_187	2008-12-31	11
-382	val_382	2008-12-31	11
-5	val_5	2008-12-31	11
-70	val_70	2008-12-31	11
-397	val_397	2008-12-31	11
-480	val_480	2008-12-31	11
-291	val_291	2008-12-31	11
-24	val_24	2008-12-31	11
-351	val_351	2008-12-31	11
-255	val_255	2008-12-31	11
-104	val_104	2008-12-31	11
-70	val_70	2008-12-31	11
-163	val_163	2008-12-31	11
-438	val_438	2008-12-31	11
-119	val_119	2008-12-31	11
-414	val_414	2008-12-31	11
-200	val_200	2008-12-31	11
-491	val_491	2008-12-31	11
-237	val_237	2008-12-31	11
-439	val_439	2008-12-31	11
-360	val_360	2008-12-31	11
-248	val_248	2008-12-31	11
-479	val_479	2008-12-31	11
-305	val_305	2008-12-31	11
-417	val_417	2008-12-31	11
-199	val_199	2008-12-31	11
-444	val_444	2008-12-31	11
-120	val_120	2008-12-31	11
-429	val_429	2008-12-31	11
-169	val_169	2008-12-31	11
-443	val_443	2008-12-31	11
-323	val_323	2008-12-31	11
-325	val_325	2008-12-31	11
-277	val_277	2008-12-31	11
-230	val_230	2008-12-31	11
-478	val_478	2008-12-31	11
-178	val_178	2008-12-31	11
-468	val_468	2008-12-31	11
-310	val_310	2008-12-31	11
-317	val_317	2008-12-31	11
-333	val_333	2008-12-31	11
-493	val_493	2008-12-31	11
-460	val_460	2008-12-31	11
-207	val_207	2008-12-31	11
-249	val_249	2008-12-31	11
-265	val_265	2008-12-31	11
-480	val_480	2008-12-31	11
-83	val_83	2008-12-31	11
-136	val_136	2008-12-31	11
-353	val_353	2008-12-31	11
-172	val_172	2008-12-31	11
-214	val_214	2008-12-31	11
-462	val_462	2008-12-31	11
-233	val_233	2008-12-31	11
-406	val_406	2008-12-31	11
-133	val_133	2008-12-31	11
-175	val_175	2008-12-31	11
-189	val_189	2008-12-31	11
-454	val_454	2008-12-31	11
-375	val_375	2008-12-31	11
-401	val_401	2008-12-31	11
-421	val_421	2008-12-31	11
-407	val_407	2008-12-31	11
-384	val_384	2008-12-31	11
-256	val_256	2008-12-31	11
-26	val_26	2008-12-31	11
-134	val_134	2008-12-31	11
-67	val_67	2008-12-31	11
-384	val_384	2008-12-31	11
-379	val_379	2008-12-31	11
-18	val_18	2008-12-31	11
-462	val_462	2008-12-31	11
-492	val_492	2008-12-31	11
-100	val_100	2008-12-31	11
-298	val_298	2008-12-31	11
-9	val_9	2008-12-31	11
-341	val_341	2008-12-31	11
-498	val_498	2008-12-31	11
-146	val_146	2008-12-31	11
-458	val_458	2008-12-31	11
-362	val_362	2008-12-31	11
-186	val_186	2008-12-31	11
-285	val_285	2008-12-31	11
-348	val_348	2008-12-31	11
-167	val_167	2008-12-31	11
-18	val_18	2008-12-31	11
-273	val_273	2008-12-31	11
-183	val_183	2008-12-31	11
-281	val_281	2008-12-31	11
-344	val_344	2008-12-31	11
-97	val_97	2008-12-31	11
-469	val_469	2008-12-31	11
-315	val_315	2008-12-31	11
-84	val_84	2008-12-31	11
-28	val_28	2008-12-31	11
-37	val_37	2008-12-31	11
-448	val_448	2008-12-31	11
-152	val_152	2008-12-31	11
-348	val_348	2008-12-31	11
-307	val_307	2008-12-31	11
-194	val_194	2008-12-31	11
-414	val_414	2008-12-31	11
-477	val_477	2008-12-31	11
-222	val_222	2008-12-31	11
-126	val_126	2008-12-31	11
-90	val_90	2008-12-31	11
-169	val_169	2008-12-31	11
-403	val_403	2008-12-31	11
-400	val_400	2008-12-31	11
-200	val_200	2008-12-31	11
-97	val_97	2008-12-31	11
-238	val_238	2008-12-31	12
-86	val_86	2008-12-31	12
-311	val_311	2008-12-31	12
-27	val_27	2008-12-31	12
-165	val_165	2008-12-31	12
-409	val_409	2008-12-31	12
-255	val_255	2008-12-31	12
-278	val_278	2008-12-31	12
-98	val_98	2008-12-31	12
-484	val_484	2008-12-31	12
-265	val_265	2008-12-31	12
-193	val_193	2008-12-31	12
-401	val_401	2008-12-31	12
-150	val_150	2008-12-31	12
-273	val_273	2008-12-31	12
-224	val_224	2008-12-31	12
-369	val_369	2008-12-31	12
-66	val_66	2008-12-31	12
-128	val_128	2008-12-31	12
-213	val_213	2008-12-31	12
-146	val_146	2008-12-31	12
-406	val_406	2008-12-31	12
-429	val_429	2008-12-31	12
-374	val_374	2008-12-31	12
-152	val_152	2008-12-31	12
-469	val_469	2008-12-31	12
-145	val_145	2008-12-31	12
-495	val_495	2008-12-31	12
-37	val_37	2008-12-31	12
-327	val_327	2008-12-31	12
-281	val_281	2008-12-31	12
-277	val_277	2008-12-31	12
-209	val_209	2008-12-31	12
-15	val_15	2008-12-31	12
-82	val_82	2008-12-31	12
-403	val_403	2008-12-31	12
-166	val_166	2008-12-31	12
-417	val_417	2008-12-31	12
-430	val_430	2008-12-31	12
-252	val_252	2008-12-31	12
-292	val_292	2008-12-31	12
-219	val_219	2008-12-31	12
-287	val_287	2008-12-31	12
-153	val_153	2008-12-31	12
-193	val_193	2008-12-31	12
-338	val_338	2008-12-31	12
-446	val_446	2008-12-31	12
-459	val_459	2008-12-31	12
-394	val_394	2008-12-31	12
-237	val_237	2008-12-31	12
-482	val_482	2008-12-31	12
-174	val_174	2008-12-31	12
-413	val_413	2008-12-31	12
-494	val_494	2008-12-31	12
-207	val_207	2008-12-31	12
-199	val_199	2008-12-31	12
-466	val_466	2008-12-31	12
-208	val_208	2008-12-31	12
-174	val_174	2008-12-31	12
-399	val_399	2008-12-31	12
-396	val_396	2008-12-31	12
-247	val_247	2008-12-31	12
-417	val_417	2008-12-31	12
-489	val_489	2008-12-31	12
-162	val_162	2008-12-31	12
-377	val_377	2008-12-31	12
-397	val_397	2008-12-31	12
-309	val_309	2008-12-31	12
-365	val_365	2008-12-31	12
-266	val_266	2008-12-31	12
-439	val_439	2008-12-31	12
-342	val_342	2008-12-31	12
-367	val_367	2008-12-31	12
-325	val_325	2008-12-31	12
-167	val_167	2008-12-31	12
-195	val_195	2008-12-31	12
-475	val_475	2008-12-31	12
-17	val_17	2008-12-31	12
-113	val_113	2008-12-31	12
-155	val_155	2008-12-31	12
-203	val_203	2008-12-31	12
-339	val_339	2008-12-31	12
-0	val_0	2008-12-31	12
-455	val_455	2008-12-31	12
-128	val_128	2008-12-31	12
-311	val_311	2008-12-31	12
-316	val_316	2008-12-31	12
-57	val_57	2008-12-31	12
-302	val_302	2008-12-31	12
-205	val_205	2008-12-31	12
-149	val_149	2008-12-31	12
-438	val_438	2008-12-31	12
-345	val_345	2008-12-31	12
-129	val_129	2008-12-31	12
-170	val_170	2008-12-31	12
-20	val_20	2008-12-31	12
-489	val_489	2008-12-31	12
-157	val_157	2008-12-31	12
-378	val_378	2008-12-31	12
-221	val_221	2008-12-31	12
-92	val_92	2008-12-31	12
-111	val_111	2008-12-31	12
-47	val_47	2008-12-31	12
-72	val_72	2008-12-31	12
-4	val_4	2008-12-31	12
-280	val_280	2008-12-31	12
-35	val_35	2008-12-31	12
-427	val_427	2008-12-31	12
-277	val_277	2008-12-31	12
-208	val_208	2008-12-31	12
-356	val_356	2008-12-31	12
-399	val_399	2008-12-31	12
-169	val_169	2008-12-31	12
-382	val_382	2008-12-31	12
-498	val_498	2008-12-31	12
-125	val_125	2008-12-31	12
-386	val_386	2008-12-31	12
-437	val_437	2008-12-31	12
-469	val_469	2008-12-31	12
-192	val_192	2008-12-31	12
-286	val_286	2008-12-31	12
-187	val_187	2008-12-31	12
-176	val_176	2008-12-31	12
-54	val_54	2008-12-31	12
-459	val_459	2008-12-31	12
-51	val_51	2008-12-31	12
-138	val_138	2008-12-31	12
-103	val_103	2008-12-31	12
-239	val_239	2008-12-31	12
-213	val_213	2008-12-31	12
-216	val_216	2008-12-31	12
-430	val_430	2008-12-31	12
-278	val_278	2008-12-31	12
-176	val_176	2008-12-31	12
-289	val_289	2008-12-31	12
-221	val_221	2008-12-31	12
-65	val_65	2008-12-31	12
-318	val_318	2008-12-31	12
-332	val_332	2008-12-31	12
-311	val_311	2008-12-31	12
-275	val_275	2008-12-31	12
-137	val_137	2008-12-31	12
-241	val_241	2008-12-31	12
-83	val_83	2008-12-31	12
-333	val_333	2008-12-31	12
-180	val_180	2008-12-31	12
-284	val_284	2008-12-31	12
-12	val_12	2008-12-31	12
-230	val_230	2008-12-31	12
-181	val_181	2008-12-31	12
-67	val_67	2008-12-31	12
-260	val_260	2008-12-31	12
-404	val_404	2008-12-31	12
-384	val_384	2008-12-31	12
-489	val_489	2008-12-31	12
-353	val_353	2008-12-31	12
-373	val_373	2008-12-31	12
-272	val_272	2008-12-31	12
-138	val_138	2008-12-31	12
-217	val_217	2008-12-31	12
-84	val_84	2008-12-31	12
-348	val_348	2008-12-31	12
-466	val_466	2008-12-31	12
-58	val_58	2008-12-31	12
-8	val_8	2008-12-31	12
-411	val_411	2008-12-31	12
-230	val_230	2008-12-31	12
-208	val_208	2008-12-31	12
-348	val_348	2008-12-31	12
-24	val_24	2008-12-31	12
-463	val_463	2008-12-31	12
-431	val_431	2008-12-31	12
-179	val_179	2008-12-31	12
-172	val_172	2008-12-31	12
-42	val_42	2008-12-31	12
-129	val_129	2008-12-31	12
-158	val_158	2008-12-31	12
-119	val_119	2008-12-31	12
-496	val_496	2008-12-31	12
-0	val_0	2008-12-31	12
-322	val_322	2008-12-31	12
-197	val_197	2008-12-31	12
-468	val_468	2008-12-31	12
-393	val_393	2008-12-31	12
-454	val_454	2008-12-31	12
-100	val_100	2008-12-31	12
-298	val_298	2008-12-31	12
-199	val_199	2008-12-31	12
-191	val_191	2008-12-31	12
-418	val_418	2008-12-31	12
-96	val_96	2008-12-31	12
-26	val_26	2008-12-31	12
-165	val_165	2008-12-31	12
-327	val_327	2008-12-31	12
-230	val_230	2008-12-31	12
-205	val_205	2008-12-31	12
-120	val_120	2008-12-31	12
-131	val_131	2008-12-31	12
-51	val_51	2008-12-31	12
-404	val_404	2008-12-31	12
-43	val_43	2008-12-31	12
-436	val_436	2008-12-31	12
-156	val_156	2008-12-31	12
-469	val_469	2008-12-31	12
-468	val_468	2008-12-31	12
-308	val_308	2008-12-31	12
-95	val_95	2008-12-31	12
-196	val_196	2008-12-31	12
-288	val_288	2008-12-31	12
-481	val_481	2008-12-31	12
-457	val_457	2008-12-31	12
-98	val_98	2008-12-31	12
-282	val_282	2008-12-31	12
-197	val_197	2008-12-31	12
-187	val_187	2008-12-31	12
-318	val_318	2008-12-31	12
-318	val_318	2008-12-31	12
-409	val_409	2008-12-31	12
-470	val_470	2008-12-31	12
-137	val_137	2008-12-31	12
-369	val_369	2008-12-31	12
-316	val_316	2008-12-31	12
-169	val_169	2008-12-31	12
-413	val_413	2008-12-31	12
-85	val_85	2008-12-31	12
-77	val_77	2008-12-31	12
-0	val_0	2008-12-31	12
-490	val_490	2008-12-31	12
-87	val_87	2008-12-31	12
-364	val_364	2008-12-31	12
-179	val_179	2008-12-31	12
-118	val_118	2008-12-31	12
-134	val_134	2008-12-31	12
-395	val_395	2008-12-31	12
-282	val_282	2008-12-31	12
-138	val_138	2008-12-31	12
-238	val_238	2008-12-31	12
-419	val_419	2008-12-31	12
-15	val_15	2008-12-31	12
-118	val_118	2008-12-31	12
-72	val_72	2008-12-31	12
-90	val_90	2008-12-31	12
-307	val_307	2008-12-31	12
-19	val_19	2008-12-31	12
-435	val_435	2008-12-31	12
-10	val_10	2008-12-31	12
-277	val_277	2008-12-31	12
-273	val_273	2008-12-31	12
-306	val_306	2008-12-31	12
-224	val_224	2008-12-31	12
-309	val_309	2008-12-31	12
-389	val_389	2008-12-31	12
-327	val_327	2008-12-31	12
-242	val_242	2008-12-31	12
-369	val_369	2008-12-31	12
-392	val_392	2008-12-31	12
-272	val_272	2008-12-31	12
-331	val_331	2008-12-31	12
-401	val_401	2008-12-31	12
-242	val_242	2008-12-31	12
-452	val_452	2008-12-31	12
-177	val_177	2008-12-31	12
-226	val_226	2008-12-31	12
-5	val_5	2008-12-31	12
-497	val_497	2008-12-31	12
-402	val_402	2008-12-31	12
-396	val_396	2008-12-31	12
-317	val_317	2008-12-31	12
-395	val_395	2008-12-31	12
-58	val_58	2008-12-31	12
-35	val_35	2008-12-31	12
-336	val_336	2008-12-31	12
-95	val_95	2008-12-31	12
-11	val_11	2008-12-31	12
-168	val_168	2008-12-31	12
-34	val_34	2008-12-31	12
-229	val_229	2008-12-31	12
-233	val_233	2008-12-31	12
-143	val_143	2008-12-31	12
-472	val_472	2008-12-31	12
-322	val_322	2008-12-31	12
-498	val_498	2008-12-31	12
-160	val_160	2008-12-31	12
-195	val_195	2008-12-31	12
-42	val_42	2008-12-31	12
-321	val_321	2008-12-31	12
-430	val_430	2008-12-31	12
-119	val_119	2008-12-31	12
-489	val_489	2008-12-31	12
-458	val_458	2008-12-31	12
-78	val_78	2008-12-31	12
-76	val_76	2008-12-31	12
-41	val_41	2008-12-31	12
-223	val_223	2008-12-31	12
-492	val_492	2008-12-31	12
-149	val_149	2008-12-31	12
-449	val_449	2008-12-31	12
-218	val_218	2008-12-31	12
-228	val_228	2008-12-31	12
-138	val_138	2008-12-31	12
-453	val_453	2008-12-31	12
-30	val_30	2008-12-31	12
-209	val_209	2008-12-31	12
-64	val_64	2008-12-31	12
-468	val_468	2008-12-31	12
-76	val_76	2008-12-31	12
-74	val_74	2008-12-31	12
-342	val_342	2008-12-31	12
-69	val_69	2008-12-31	12
-230	val_230	2008-12-31	12
-33	val_33	2008-12-31	12
-368	val_368	2008-12-31	12
-103	val_103	2008-12-31	12
-296	val_296	2008-12-31	12
-113	val_113	2008-12-31	12
-216	val_216	2008-12-31	12
-367	val_367	2008-12-31	12
-344	val_344	2008-12-31	12
-167	val_167	2008-12-31	12
-274	val_274	2008-12-31	12
-219	val_219	2008-12-31	12
-239	val_239	2008-12-31	12
-485	val_485	2008-12-31	12
-116	val_116	2008-12-31	12
-223	val_223	2008-12-31	12
-256	val_256	2008-12-31	12
-263	val_263	2008-12-31	12
-70	val_70	2008-12-31	12
-487	val_487	2008-12-31	12
-480	val_480	2008-12-31	12
-401	val_401	2008-12-31	12
-288	val_288	2008-12-31	12
-191	val_191	2008-12-31	12
-5	val_5	2008-12-31	12
-244	val_244	2008-12-31	12
-438	val_438	2008-12-31	12
-128	val_128	2008-12-31	12
-467	val_467	2008-12-31	12
-432	val_432	2008-12-31	12
-202	val_202	2008-12-31	12
-316	val_316	2008-12-31	12
-229	val_229	2008-12-31	12
-469	val_469	2008-12-31	12
-463	val_463	2008-12-31	12
-280	val_280	2008-12-31	12
-2	val_2	2008-12-31	12
-35	val_35	2008-12-31	12
-283	val_283	2008-12-31	12
-331	val_331	2008-12-31	12
-235	val_235	2008-12-31	12
-80	val_80	2008-12-31	12
-44	val_44	2008-12-31	12
-193	val_193	2008-12-31	12
-321	val_321	2008-12-31	12
-335	val_335	2008-12-31	12
-104	val_104	2008-12-31	12
-466	val_466	2008-12-31	12
-366	val_366	2008-12-31	12
-175	val_175	2008-12-31	12
-403	val_403	2008-12-31	12
-483	val_483	2008-12-31	12
-53	val_53	2008-12-31	12
-105	val_105	2008-12-31	12
-257	val_257	2008-12-31	12
-406	val_406	2008-12-31	12
-409	val_409	2008-12-31	12
-190	val_190	2008-12-31	12
-406	val_406	2008-12-31	12
-401	val_401	2008-12-31	12
-114	val_114	2008-12-31	12
-258	val_258	2008-12-31	12
-90	val_90	2008-12-31	12
-203	val_203	2008-12-31	12
-262	val_262	2008-12-31	12
-348	val_348	2008-12-31	12
-424	val_424	2008-12-31	12
-12	val_12	2008-12-31	12
-396	val_396	2008-12-31	12
-201	val_201	2008-12-31	12
-217	val_217	2008-12-31	12
-164	val_164	2008-12-31	12
-431	val_431	2008-12-31	12
-454	val_454	2008-12-31	12
-478	val_478	2008-12-31	12
-298	val_298	2008-12-31	12
-125	val_125	2008-12-31	12
-431	val_431	2008-12-31	12
-164	val_164	2008-12-31	12
-424	val_424	2008-12-31	12
-187	val_187	2008-12-31	12
-382	val_382	2008-12-31	12
-5	val_5	2008-12-31	12
-70	val_70	2008-12-31	12
-397	val_397	2008-12-31	12
-480	val_480	2008-12-31	12
-291	val_291	2008-12-31	12
-24	val_24	2008-12-31	12
-351	val_351	2008-12-31	12
-255	val_255	2008-12-31	12
-104	val_104	2008-12-31	12
-70	val_70	2008-12-31	12
-163	val_163	2008-12-31	12
-438	val_438	2008-12-31	12
-119	val_119	2008-12-31	12
-414	val_414	2008-12-31	12
-200	val_200	2008-12-31	12
-491	val_491	2008-12-31	12
-237	val_237	2008-12-31	12
-439	val_439	2008-12-31	12
-360	val_360	2008-12-31	12
-248	val_248	2008-12-31	12
-479	val_479	2008-12-31	12
-305	val_305	2008-12-31	12
-417	val_417	2008-12-31	12
-199	val_199	2008-12-31	12
-444	val_444	2008-12-31	12
-120	val_120	2008-12-31	12
-429	val_429	2008-12-31	12
-169	val_169	2008-12-31	12
-443	val_443	2008-12-31	12
-323	val_323	2008-12-31	12
-325	val_325	2008-12-31	12
-277	val_277	2008-12-31	12
-230	val_230	2008-12-31	12
-478	val_478	2008-12-31	12
-178	val_178	2008-12-31	12
-468	val_468	2008-12-31	12
-310	val_310	2008-12-31	12
-317	val_317	2008-12-31	12
-333	val_333	2008-12-31	12
-493	val_493	2008-12-31	12
-460	val_460	2008-12-31	12
-207	val_207	2008-12-31	12
-249	val_249	2008-12-31	12
-265	val_265	2008-12-31	12
-480	val_480	2008-12-31	12
-83	val_83	2008-12-31	12
-136	val_136	2008-12-31	12
-353	val_353	2008-12-31	12
-172	val_172	2008-12-31	12
-214	val_214	2008-12-31	12
-462	val_462	2008-12-31	12
-233	val_233	2008-12-31	12
-406	val_406	2008-12-31	12
-133	val_133	2008-12-31	12
-175	val_175	2008-12-31	12
-189	val_189	2008-12-31	12
-454	val_454	2008-12-31	12
-375	val_375	2008-12-31	12
-401	val_401	2008-12-31	12
-421	val_421	2008-12-31	12
-407	val_407	2008-12-31	12
-384	val_384	2008-12-31	12
-256	val_256	2008-12-31	12
-26	val_26	2008-12-31	12
-134	val_134	2008-12-31	12
-67	val_67	2008-12-31	12
-384	val_384	2008-12-31	12
-379	val_379	2008-12-31	12
-18	val_18	2008-12-31	12
-462	val_462	2008-12-31	12
-492	val_492	2008-12-31	12
-100	val_100	2008-12-31	12
-298	val_298	2008-12-31	12
-9	val_9	2008-12-31	12
-341	val_341	2008-12-31	12
-498	val_498	2008-12-31	12
-146	val_146	2008-12-31	12
-458	val_458	2008-12-31	12
-362	val_362	2008-12-31	12
-186	val_186	2008-12-31	12
-285	val_285	2008-12-31	12
-348	val_348	2008-12-31	12
-167	val_167	2008-12-31	12
-18	val_18	2008-12-31	12
-273	val_273	2008-12-31	12
-183	val_183	2008-12-31	12
-281	val_281	2008-12-31	12
-344	val_344	2008-12-31	12
-97	val_97	2008-12-31	12
-469	val_469	2008-12-31	12
-315	val_315	2008-12-31	12
-84	val_84	2008-12-31	12
-28	val_28	2008-12-31	12
-37	val_37	2008-12-31	12
-448	val_448	2008-12-31	12
-152	val_152	2008-12-31	12
-348	val_348	2008-12-31	12
-307	val_307	2008-12-31	12
-194	val_194	2008-12-31	12
-414	val_414	2008-12-31	12
-477	val_477	2008-12-31	12
-222	val_222	2008-12-31	12
-126	val_126	2008-12-31	12
-90	val_90	2008-12-31	12
-169	val_169	2008-12-31	12
-403	val_403	2008-12-31	12
-400	val_400	2008-12-31	12
-200	val_200	2008-12-31	12
-97	val_97	2008-12-31	12
diff --git a/sql/hive/src/test/resources/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c b/sql/hive/src/test/resources/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c
deleted file mode 100644
index 2ac44ee23f45c47ec2e5ee60669652d754cab497..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 11]    	 
-Database:           	default             	 
-Table:              	nzhang_part1        	 
-CreateTime:         	Tue Jan 28 02:18:26 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part1/ds=2008-04-08/hr=11	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904306          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 b/sql/hive/src/test/resources/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70
deleted file mode 100644
index f0b07ca89fcb37a86a319eedef6d3e9d4f33c5e9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 12]    	 
-Database:           	default             	 
-Table:              	nzhang_part1        	 
-CreateTime:         	Tue Jan 28 02:18:26 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part1/ds=2008-04-08/hr=12	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904306          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 b/sql/hive/src/test/resources/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9
deleted file mode 100644
index f94614124bcf619c06e365bae78cb2c8401480af..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-12-31, 11]    	 
-Database:           	default             	 
-Table:              	nzhang_part2        	 
-CreateTime:         	Tue Jan 28 02:18:26 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part2/ds=2008-12-31/hr=11	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904306          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 b/sql/hive/src/test/resources/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7
deleted file mode 100644
index f00fb5b3bd49131723c58389752c43106381c408..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-12-31, 12]    	 
-Database:           	default             	 
-Table:              	nzhang_part2        	 
-CreateTime:         	Tue Jan 28 02:18:26 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part2/ds=2008-12-31/hr=12	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904306          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 b/sql/hive/src/test/resources/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5
deleted file mode 100644
index 7335c8d32fa68bd052938f8a76cf57165f41c52f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5
+++ /dev/null
@@ -1,39 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 28 02:18:17 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	2                   
-	numPartitions       	2                   
-	numRows             	1000                
-	p3                  	v3                  
-	rawDataSize         	10624               
-	totalSize           	11624               
-	transient_lastDdlTime	1390904306          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats4-20-f63000f2c395b935199c9829964f98c1 b/sql/hive/src/test/resources/golden/stats4-20-f63000f2c395b935199c9829964f98c1
deleted file mode 100644
index 674cd9b0b64e5327f5659f19007f24715e83e495..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats4-20-f63000f2c395b935199c9829964f98c1
+++ /dev/null
@@ -1,39 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 28 02:18:17 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part2	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	2                   
-	numPartitions       	2                   
-	numRows             	1000                
-	p3                  	v3                  
-	rawDataSize         	10624               
-	totalSize           	11624               
-	transient_lastDdlTime	1390904306          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd b/sql/hive/src/test/resources/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c b/sql/hive/src/test/resources/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd b/sql/hive/src/test/resources/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c b/sql/hive/src/test/resources/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d b/sql/hive/src/test/resources/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 b/sql/hive/src/test/resources/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f b/sql/hive/src/test/resources/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 b/sql/hive/src/test/resources/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats5-3-96d9aa9c32a081518604959dcfac42df b/sql/hive/src/test/resources/golden/stats5-3-96d9aa9c32a081518604959dcfac42df
deleted file mode 100644
index c9a75eb6639d4c436a404b088b208d753c30bb2f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats5-3-96d9aa9c32a081518604959dcfac42df
+++ /dev/null
@@ -1,19 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_src)))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 depends on stages: Stage-0
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        analyze_src 
-          TableScan
-            alias: analyze_src
-
-  Stage: Stage-1
-    Stats-Aggr Operator
-
-
diff --git a/sql/hive/src/test/resources/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 b/sql/hive/src/test/resources/golden/stats5-4-dbf81a12f6c19c14dce831e942870744
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 b/sql/hive/src/test/resources/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361
deleted file mode 100644
index e622a0d3704a16804df93f43ac7c7a40e7ed3c61..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361
+++ /dev/null
@@ -1,33 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 28 02:18:36 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_src	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	1                   
-	numPartitions       	0                   
-	numRows             	500                 
-	p3                  	v3                  
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904324          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats6-10-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats6-10-653f748fa2d690502ba4fda407841a20
deleted file mode 100644
index bf589ab89431217e10aa3898b2a3b03af96ed1d5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats6-10-653f748fa2d690502ba4fda407841a20
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-09, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 28 02:18:52 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=11	 
-Partition Parameters:	 	 
-	transient_lastDdlTime	1390904332          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats6-11-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats6-11-1c9f833953403596ad50fd32e513642c
deleted file mode 100644
index e6170a33d7de7b95455d031c230fb752941c4a67..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats6-11-1c9f833953403596ad50fd32e513642c
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-09, 12]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 28 02:18:52 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=12	 
-Partition Parameters:	 	 
-	transient_lastDdlTime	1390904332          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492
deleted file mode 100644
index 45f6b344b04ca9cd350668df54cd3d6313925c82..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492
+++ /dev/null
@@ -1,39 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 28 02:18:45 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	2                   
-	numPartitions       	2                   
-	numRows             	1000                
-	p3                  	v3                  
-	rawDataSize         	10624               
-	totalSize           	11624               
-	transient_lastDdlTime	1390904348          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats6-4-a88c476a632cd92f748967fadb242405 b/sql/hive/src/test/resources/golden/stats6-4-a88c476a632cd92f748967fadb242405
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats6-6-8926095434b70c83bf88c70559d38dce b/sql/hive/src/test/resources/golden/stats6-6-8926095434b70c83bf88c70559d38dce
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 b/sql/hive/src/test/resources/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7
deleted file mode 100644
index 9ff40db416cb9348400a27d5a778f0c39012d5c8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 28 02:18:52 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=11	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904340          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d
deleted file mode 100644
index 5c13e8793cc76fab0fed65e06d09ecac8962097d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 12]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 28 02:18:52 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=12	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904348          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492
deleted file mode 100644
index bafa551db56e9e3adca47bcddf72d7f191ed8dfa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492
+++ /dev/null
@@ -1,38 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 14 14:57:42 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	2                   
-	numPartitions       	2                   
-	numRows             	1000                
-	rawDataSize         	10624               
-	totalSize           	11624               
-	transient_lastDdlTime	1389740280          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats7-4-a88c476a632cd92f748967fadb242405 b/sql/hive/src/test/resources/golden/stats7-4-a88c476a632cd92f748967fadb242405
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 b/sql/hive/src/test/resources/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2
deleted file mode 100644
index d5341825cff70648315647011ceff798a39f50d8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2
+++ /dev/null
@@ -1,18 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr))))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 depends on stages: Stage-0
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        analyze_srcpart 
-          TableScan
-            alias: analyze_srcpart
-
-  Stage: Stage-1
-    Stats-Aggr Operator
-
diff --git a/sql/hive/src/test/resources/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 b/sql/hive/src/test/resources/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7
deleted file mode 100644
index 7085e9397e59bb34cfe0d80052649e14eb573879..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 14 14:57:50 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-08/hr=11	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1389740280          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d
deleted file mode 100644
index a3eb8c6466032593c9f5cf5745408facef8f34ec..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 12]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 14 14:57:50 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-08/hr=12	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1389740280          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 b/sql/hive/src/test/resources/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8
deleted file mode 100644
index d1e0c6243244d7de39d3e78ffd3599f39fdc5094..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8
+++ /dev/null
@@ -1,19 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 12))))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 depends on stages: Stage-0
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        analyze_srcpart 
-          TableScan
-            alias: analyze_srcpart
-
-  Stage: Stage-1
-    Stats-Aggr Operator
-
-
diff --git a/sql/hive/src/test/resources/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 b/sql/hive/src/test/resources/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d
deleted file mode 100644
index 27ff9a687f5c6f266c8d59f92ef82027d8b6edd3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 12]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 28 02:19:18 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=12	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904374          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d b/sql/hive/src/test/resources/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d
deleted file mode 100644
index ce340cced8844879abe5e9dd37a0604a9f3be8bc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d
+++ /dev/null
@@ -1,19 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-09') (TOK_PARTVAL hr 11))))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 depends on stages: Stage-0
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        analyze_srcpart 
-          TableScan
-            alias: analyze_srcpart
-
-  Stage: Stage-1
-    Stats-Aggr Operator
-
-
diff --git a/sql/hive/src/test/resources/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 b/sql/hive/src/test/resources/golden/stats8-14-ea9afc1343991ed4d410231803a174f7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats8-15-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats8-15-653f748fa2d690502ba4fda407841a20
deleted file mode 100644
index 649828a63174b7ab21638ffb8357521e93578257..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-15-653f748fa2d690502ba4fda407841a20
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-09, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 28 02:19:18 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=11	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904382          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 b/sql/hive/src/test/resources/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6
deleted file mode 100644
index e4c8bf3e0005f60cf812511f60efce335b845574..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6
+++ /dev/null
@@ -1,19 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-09') (TOK_PARTVAL hr 12))))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 depends on stages: Stage-0
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        analyze_srcpart 
-          TableScan
-            alias: analyze_srcpart
-
-  Stage: Stage-1
-    Stats-Aggr Operator
-
-
diff --git a/sql/hive/src/test/resources/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 b/sql/hive/src/test/resources/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats8-18-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats8-18-1c9f833953403596ad50fd32e513642c
deleted file mode 100644
index 74aeceec373989866761a0e90202a9eab0e846df..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-18-1c9f833953403596ad50fd32e513642c
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-09, 12]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 28 02:19:18 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=12	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904390          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 b/sql/hive/src/test/resources/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16
deleted file mode 100644
index 25d9c86f74634b590a416bb4b2cdaf37ae830a56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16
+++ /dev/null
@@ -1,19 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds) (TOK_PARTVAL hr))))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 depends on stages: Stage-0
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        analyze_srcpart 
-          TableScan
-            alias: analyze_srcpart
-
-  Stage: Stage-1
-    Stats-Aggr Operator
-
-
diff --git a/sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 b/sql/hive/src/test/resources/golden/stats8-20-bff592750d1057448d2cff7694b6dad2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7
deleted file mode 100644
index 5939c257f2f672708b64bc66e79d077c0b44042a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 28 02:19:18 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=11	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904398          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d
deleted file mode 100644
index 8dc22de8e0ee6c250ac26e913949e88727be56c6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 12]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 28 02:19:18 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=12	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904398          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats8-23-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats8-23-653f748fa2d690502ba4fda407841a20
deleted file mode 100644
index a19f38cc6a364f78cb2e8027c2cdbad82c400ec1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-23-653f748fa2d690502ba4fda407841a20
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-09, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 28 02:19:18 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=11	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904398          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats8-24-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats8-24-1c9f833953403596ad50fd32e513642c
deleted file mode 100644
index aead24acd518c6af6939a4a2d9bdf207911fe565..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-24-1c9f833953403596ad50fd32e513642c
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-09, 12]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 28 02:19:18 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=12	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904398          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492
deleted file mode 100644
index 5f8bf1774f1448f373e8b52ffb43ee8d233afe7a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492
+++ /dev/null
@@ -1,39 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 28 02:19:10 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	4                   
-	numPartitions       	4                   
-	numRows             	2000                
-	p3                  	v3                  
-	rawDataSize         	21248               
-	totalSize           	23248               
-	transient_lastDdlTime	1390904398          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats8-4-a88c476a632cd92f748967fadb242405 b/sql/hive/src/test/resources/golden/stats8-4-a88c476a632cd92f748967fadb242405
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 b/sql/hive/src/test/resources/golden/stats8-6-821e1f061960081b9b379d9bfb47f267
deleted file mode 100644
index d30acbf86a29584c8b3808e2ac7488ff284e6741..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-6-821e1f061960081b9b379d9bfb47f267
+++ /dev/null
@@ -1,19 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 11))))
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-  Stage-1 depends on stages: Stage-0
-
-STAGE PLANS:
-  Stage: Stage-0
-    Map Reduce
-      Alias -> Map Operator Tree:
-        analyze_srcpart 
-          TableScan
-            alias: analyze_srcpart
-
-  Stage: Stage-1
-    Stats-Aggr Operator
-
-
diff --git a/sql/hive/src/test/resources/golden/stats8-7-8926095434b70c83bf88c70559d38dce b/sql/hive/src/test/resources/golden/stats8-7-8926095434b70c83bf88c70559d38dce
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7
deleted file mode 100644
index 07a61fc1a8bd14e6984d5033c67752bd3d582d64..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 28 02:19:18 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=11	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904366          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492
deleted file mode 100644
index bd16e76d28d83dc3e205824e7cf944c3ffa5e4f4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492
+++ /dev/null
@@ -1,39 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 28 02:19:10 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	1                   
-	numPartitions       	1                   
-	numRows             	500                 
-	p3                  	v3                  
-	rawDataSize         	5312                
-	totalSize           	5812                
-	transient_lastDdlTime	1390904366          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6f3df708fa339df236ec9375d2ad37fa
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c
rename to sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6f3df708fa339df236ec9375d2ad37fa
diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-36eee5cbac5c0c3228e499805b32f6
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-36eee5cbac5c0c3228e499805b32f6
diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6
+++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb
+++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6
+++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614
+++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6
+++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6
+++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1
+++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3
+++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3
+++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6
+++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3
+++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0
+++ b/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 b/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105
+++ b/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 b/sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55
index 4475cb837eaa4d4aecb5b49389899ed16415cd24..e01e7187693ed5f8f484013ca34ff0061c35ed6b 100644
--- a/sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55
+++ b/sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55
@@ -1,27 +1,28 @@
 # col_name            	data_type           	comment             
 	 	 
-key                 	string              	None                
-value               	string              	None                
+key                 	string              	                    
+value               	string              	                    
 	 	 
 # Partition Information	 	 
 # col_name            	data_type           	comment             
 	 	 
-part                	string              	None                
+part                	string              	                    
 	 	 
 # Detailed Partition Information	 	 
 Partition Value:    	[1]                 	 
 Database:           	default             	 
 Table:              	tmptable            	 
-CreateTime:         	Sun Jan 05 00:32:00 PST 2014	 
+CreateTime:         	Tue Oct 21 05:20:33 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2020775722466758355/tmptable/part=1	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmptable/part=1	 
 Partition Parameters:	 	 
+	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
 	numRows             	0                   
 	rawDataSize         	0                   
 	totalSize           	0                   
-	transient_lastDdlTime	1388910720          
+	transient_lastDdlTime	1413894033          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
@@ -32,4 +33,4 @@ Num Buckets:        	-1
 Bucket Columns:     	[]                  	 
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d
deleted file mode 100644
index 5949a76d84a11c9d9e7df990f6dbd4c009ed918c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 12]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 14 14:58:14 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-08/hr=12	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	0                   
-	rawDataSize         	0                   
-	totalSize           	5812                
-	transient_lastDdlTime	1389740295          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20
deleted file mode 100644
index 97f5929fd816a17007d456b3218a2980439ab449..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-09, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 14 14:58:14 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-09/hr=11	 
-Partition Parameters:	 	 
-	transient_lastDdlTime	1389740294          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c
deleted file mode 100644
index 6d08ff47abc2cbd3b4dbbb8f5d56adf27d6d7f21..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-09, 12]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 14 14:58:14 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-09/hr=12	 
-Partition Parameters:	 	 
-	transient_lastDdlTime	1389740294          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492
deleted file mode 100644
index f441c8b4838688ddb7ef6ce6139df3d1ca6cfbc1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492
+++ /dev/null
@@ -1,38 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 14 14:58:04 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	numFiles            	2                   
-	numPartitions       	2                   
-	numRows             	0                   
-	rawDataSize         	0                   
-	totalSize           	11624               
-	transient_lastDdlTime	1389740295          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab b/sql/hive/src/test/resources/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 b/sql/hive/src/test/resources/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 b/sql/hive/src/test/resources/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 b/sql/hive/src/test/resources/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3
deleted file mode 100644
index f263eed117b104c2280abc920a904b5cc40db8fc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3
+++ /dev/null
@@ -1,10 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart_partial) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08'))) noscan)
-
-STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-1
-    Stats-Aggr Operator
-
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 b/sql/hive/src/test/resources/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e b/sql/hive/src/test/resources/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e
deleted file mode 100644
index 6a115f7e6e61ae9dd7a26fa39d7f1560fa624fb7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart_partial	 
-CreateTime:         	Tue Jan 14 14:58:26 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart_partial/ds=2008-04-08/hr=11	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	0                   
-	rawDataSize         	0                   
-	totalSize           	5812                
-	transient_lastDdlTime	1389740307          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f b/sql/hive/src/test/resources/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f
deleted file mode 100644
index 1d96413b915da5bbbe891f4eaf89ed635a829ede..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 12]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart_partial	 
-CreateTime:         	Tue Jan 14 14:58:26 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart_partial/ds=2008-04-08/hr=12	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	0                   
-	rawDataSize         	0                   
-	totalSize           	5812                
-	transient_lastDdlTime	1389740307          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 b/sql/hive/src/test/resources/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709
deleted file mode 100644
index 4c4380e63b7f979821efbdfaffa354b21e949919..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-09, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart_partial	 
-CreateTime:         	Tue Jan 14 14:58:27 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart_partial/ds=2008-04-09/hr=11	 
-Partition Parameters:	 	 
-	transient_lastDdlTime	1389740307          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a b/sql/hive/src/test/resources/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a
deleted file mode 100644
index ba90d8d3d24770a6f3ae922a9aa52a93cfea6299..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-09, 12]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart_partial	 
-CreateTime:         	Tue Jan 14 14:58:27 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart_partial/ds=2008-04-09/hr=12	 
-Partition Parameters:	 	 
-	transient_lastDdlTime	1389740307          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 b/sql/hive/src/test/resources/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c b/sql/hive/src/test/resources/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 b/sql/hive/src/test/resources/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670
deleted file mode 100644
index b671e68cc4c3b97082f990821cbe7dcd00d22495..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670
+++ /dev/null
@@ -1,10 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 11))) noscan)
-
-STAGE DEPENDENCIES:
-  Stage-1 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-1
-    Stats-Aggr Operator
-
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e b/sql/hive/src/test/resources/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae b/sql/hive/src/test/resources/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 b/sql/hive/src/test/resources/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44
deleted file mode 100644
index 96499c0fb0f7eafac3887c302aca5d3a6887a35f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	int                 	None                
-value               	string              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart     	 
-CreateTime:         	Tue Jan 14 14:58:14 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-08/hr=11	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	0                   
-	rawDataSize         	0                   
-	totalSize           	5812                
-	transient_lastDdlTime	1389740294          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 b/sql/hive/src/test/resources/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380
deleted file mode 100644
index b5fc469438c8360b3c67f43c2e52b4ffc915f1a6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-value               	string              	from deserializer   
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart_partial_scan	 
-CreateTime:         	Tue Jan 14 14:36:56 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2847673635801001933/analyze_srcpart_partial_scan/ds=2008-04-08/hr=11	 
-Partition Parameters:	 	 
-	transient_lastDdlTime	1389739016          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef b/sql/hive/src/test/resources/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef
deleted file mode 100644
index cc58ef9026786b3682a7e462dca827d5d7d32b25..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef
+++ /dev/null
@@ -1,14 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart_partial_scan) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 11))) partialscan)
-
-STAGE DEPENDENCIES:
-  Stage-2 is a root stage
-  Stage-1 depends on stages: Stage-2
-
-STAGE PLANS:
-  Stage: Stage-2
-    Partial Scan Statistics
-
-  Stage: Stage-1
-    Stats-Aggr Operator
-
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 b/sql/hive/src/test/resources/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c b/sql/hive/src/test/resources/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c
deleted file mode 100644
index 3243fe9fb497d79b9f27dcd0e483d934a9654816..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-value               	string              	from deserializer   
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart_partial_scan	 
-CreateTime:         	Tue Jan 14 14:36:56 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2847673635801001933/analyze_srcpart_partial_scan/ds=2008-04-08/hr=11	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	4812                
-	totalSize           	7456                
-	transient_lastDdlTime	1389739019          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 b/sql/hive/src/test/resources/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40
deleted file mode 100644
index 6accd64d06d691177b254edd0d7a8a8d46e737e3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-value               	string              	from deserializer   
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-09, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart_partial_scan	 
-CreateTime:         	Tue Jan 14 14:36:56 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2847673635801001933/analyze_srcpart_partial_scan/ds=2008-04-09/hr=11	 
-Partition Parameters:	 	 
-	transient_lastDdlTime	1389739016          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 b/sql/hive/src/test/resources/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 b/sql/hive/src/test/resources/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 b/sql/hive/src/test/resources/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380
deleted file mode 100644
index 942b7cfe07235932015d381a61dfc37a5df739a6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-value               	string              	from deserializer   
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart_partial_scan	 
-CreateTime:         	Tue Jan 28 02:20:21 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart_partial_scan/ds=2008-04-08/hr=11	 
-Partition Parameters:	 	 
-	transient_lastDdlTime	1390904421          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef b/sql/hive/src/test/resources/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef
deleted file mode 100644
index 9c704a6ef412655bbeda4f1aad139f2ff0715849..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef
+++ /dev/null
@@ -1,15 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart_partial_scan) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 11))) partialscan)
-
-STAGE DEPENDENCIES:
-  Stage-2 is a root stage
-  Stage-1 depends on stages: Stage-2
-
-STAGE PLANS:
-  Stage: Stage-2
-    Partial Scan Statistics
-
-  Stage: Stage-1
-    Stats-Aggr Operator
-
-
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c b/sql/hive/src/test/resources/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c
deleted file mode 100644
index feee75f095d0dd6a60465d88b276ba6aeacf69dc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c
+++ /dev/null
@@ -1,36 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-value               	string              	from deserializer   
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-08, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart_partial_scan	 
-CreateTime:         	Tue Jan 28 02:20:21 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart_partial_scan/ds=2008-04-08/hr=11	 
-Partition Parameters:	 	 
-	numFiles            	1                   
-	numRows             	500                 
-	rawDataSize         	4812                
-	totalSize           	7456                
-	transient_lastDdlTime	1390904425          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40
deleted file mode 100644
index f39d366764c9502427788f3f8a356a9b3d37136d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-value               	string              	from deserializer   
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-hr                  	string              	None                
-	 	 
-# Detailed Partition Information	 	 
-Partition Value:    	[2008-04-09, 11]    	 
-Database:           	default             	 
-Table:              	analyze_srcpart_partial_scan	 
-CreateTime:         	Tue Jan 28 02:20:21 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart_partial_scan/ds=2008-04-09/hr=11	 
-Partition Parameters:	 	 
-	transient_lastDdlTime	1390904421          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca b/sql/hive/src/test/resources/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-36eee5cbac5c0c3228e499805b32f6
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/stats_publisher_error_1-1-36eee5cbac5c0c3228e499805b32f6
diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 b/sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73
deleted file mode 100644
index 9bed96d85547667a35d3489664677522a8d1cffd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73
+++ /dev/null
@@ -1 +0,0 @@
-str_to_map(text, delimiter1, delimiter2) - Creates a map by parsing text 
diff --git a/sql/hive/src/test/resources/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c b/sql/hive/src/test/resources/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c
deleted file mode 100644
index 1da11fbfa31d5fe8f97f2e2daa7ed166ac300a83..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c
+++ /dev/null
@@ -1,2 +0,0 @@
-str_to_map(text, delimiter1, delimiter2) - Creates a map by parsing text 
-Split text into key-value pairs using two delimiters. The first delimiter seperates pairs, and the second delimiter sperates key and value. If only one parameter is given, default delimiters are used: ',' as delimiter1 and '=' as delimiter2.
diff --git a/sql/hive/src/test/resources/golden/str_to_map-10-32997010bba305ec40812df254490730 b/sql/hive/src/test/resources/golden/str_to_map-10-32997010bba305ec40812df254490730
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a b/sql/hive/src/test/resources/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e b/sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e
deleted file mode 100644
index cd4d4a384163b240ba6ead9872da2924381192f0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e
+++ /dev/null
@@ -1,3 +0,0 @@
-444
-444
-444
diff --git a/sql/hive/src/test/resources/golden/str_to_map-13-32997010bba305ec40812df254490730 b/sql/hive/src/test/resources/golden/str_to_map-13-32997010bba305ec40812df254490730
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec b/sql/hive/src/test/resources/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 b/sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66
deleted file mode 100644
index e8183f05f5db68b3934e93f4bf6bed2bb664e0b5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66
+++ /dev/null
@@ -1,3 +0,0 @@
-1
-1
-1
diff --git a/sql/hive/src/test/resources/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 b/sql/hive/src/test/resources/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda b/sql/hive/src/test/resources/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda
deleted file mode 100644
index 62813f9d104918fd66c7645b579bf90e2c7994c3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda
+++ /dev/null
@@ -1,3 +0,0 @@
-{"b":"2","a":"1","c":"3"}
-{"b":"2","a":"1","c":"3"}
-{"b":"2","a":"1","c":"3"}
diff --git a/sql/hive/src/test/resources/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf b/sql/hive/src/test/resources/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad b/sql/hive/src/test/resources/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad
deleted file mode 100644
index 62813f9d104918fd66c7645b579bf90e2c7994c3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad
+++ /dev/null
@@ -1,3 +0,0 @@
-{"b":"2","a":"1","c":"3"}
-{"b":"2","a":"1","c":"3"}
-{"b":"2","a":"1","c":"3"}
diff --git a/sql/hive/src/test/resources/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 b/sql/hive/src/test/resources/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd b/sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd
deleted file mode 100644
index e8183f05f5db68b3934e93f4bf6bed2bb664e0b5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd
+++ /dev/null
@@ -1,3 +0,0 @@
-1
-1
-1
diff --git a/sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 b/sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1
index 8938b37682023319db47fb7c03487f3d6a2188d8..cfb0572d8663d5a6b029455064e7fe495d6ae0e2 100644
--- a/sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1	
+++ b/sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1	
@@ -497,4 +497,4 @@ test
 test
 test
 test
-test
\ No newline at end of file
+test
diff --git a/sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca b/sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca
index 5b3f76a5f77970645a6cfdac220d4a0d889d4a2d..0bc48337e2dc02ada85201a35084c12b957ae1b3 100644
--- a/sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca
+++ b/sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca
@@ -255,4 +255,4 @@
 495	1
 496	1
 497	1
-498	3
\ No newline at end of file
+498	3
diff --git a/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 b/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504
+++ b/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-0-3b2fa9592648fc07c0d43e0d7d7f9411 b/sql/hive/src/test/resources/golden/symlink_text_input_format-0-3b2fa9592648fc07c0d43e0d7d7f9411
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-1-d498fb503b8f47db4741fdad3d266b4a b/sql/hive/src/test/resources/golden/symlink_text_input_format-1-d498fb503b8f47db4741fdad3d266b4a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5 b/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615 b/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-4-676cb274a770a6b9ca86df5dc7f912d4 b/sql/hive/src/test/resources/golden/symlink_text_input_format-4-676cb274a770a6b9ca86df5dc7f912d4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-5-ef3052815ec41b5957627698ba06707b b/sql/hive/src/test/resources/golden/symlink_text_input_format-5-ef3052815ec41b5957627698ba06707b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-6-891be0baec05e358a647dcca77724446 b/sql/hive/src/test/resources/golden/symlink_text_input_format-6-891be0baec05e358a647dcca77724446
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-7-208bcc9c918cbeb52907c8871be19cd5 b/sql/hive/src/test/resources/golden/symlink_text_input_format-7-208bcc9c918cbeb52907c8871be19cd5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-8-fb2e7127e07ad9f7e97ad3df3eba3a35 b/sql/hive/src/test/resources/golden/symlink_text_input_format-8-fb2e7127e07ad9f7e97ad3df3eba3a35
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841 b/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e b/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e
+++ b/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 b/sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2
index 92cca3b6f1e242252a27227355bb6179289c6802..95e2ae6a0fd50b14325860a56dfadd2144278dee 100644
--- a/sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2
+++ b/sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2
@@ -1,2 +1,2 @@
-a                   	int                 	None                
-b                   	string              	None                
\ No newline at end of file
+a                   	int                 	                    
+b                   	string              	                    
diff --git a/sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 b/sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113
index 55d948599907211562da45bf68242aa0462d556f..b70e127e82d056a30f56d6141cafa419ffef0184 100644
--- a/sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113
+++ b/sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113
@@ -497,4 +497,4 @@
 497	val_497
 498	val_498
 498	val_498
-498	val_498
\ No newline at end of file
+498	val_498
diff --git a/sql/hive/src/test/resources/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 b/sql/hive/src/test/resources/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c b/sql/hive/src/test/resources/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 b/sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364
deleted file mode 100644
index d00491fd7e5bb6fa28c517a0bb32b8b506539d4d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364
+++ /dev/null
@@ -1 +0,0 @@
-1
diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/timestamp_1-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/timestamp_1-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 b/sql/hive/src/test/resources/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 b/sql/hive/src/test/resources/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 b/sql/hive/src/test/resources/golden/timestamp_1-1-d362501d0176855077e65f8faf067fa8
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5
rename to sql/hive/src/test/resources/golden/timestamp_1-1-d362501d0176855077e65f8faf067fa8
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-18-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-10-67f274bf16de625cf4e85af0c6185cac
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-18-67f274bf16de625cf4e85af0c6185cac
rename to sql/hive/src/test/resources/golden/timestamp_1-10-67f274bf16de625cf4e85af0c6185cac
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-10-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-11-343c75daac6695917608c17db8bf473e
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-10-343c75daac6695917608c17db8bf473e
rename to sql/hive/src/test/resources/golden/timestamp_1-11-343c75daac6695917608c17db8bf473e
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 b/sql/hive/src/test/resources/golden/timestamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-12-cf19f7359a6d3456c4526b2c69f92d6a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a
rename to sql/hive/src/test/resources/golden/timestamp_1-12-cf19f7359a6d3456c4526b2c69f92d6a
diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d b/sql/hive/src/test/resources/golden/timestamp_1-13-d242038c04dd4ee6075c7eebc0f75f17
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d
rename to sql/hive/src/test/resources/golden/timestamp_1-13-d242038c04dd4ee6075c7eebc0f75f17
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-14-90269c1e50c7ae8e75ca9cc297982135
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135
rename to sql/hive/src/test/resources/golden/timestamp_1-14-90269c1e50c7ae8e75ca9cc297982135
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-15-e6bfca320c4ee3aff39cf2f179d57da6
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6
rename to sql/hive/src/test/resources/golden/timestamp_1-15-e6bfca320c4ee3aff39cf2f179d57da6
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-16-d0291a9bd42054b2732cb4f54cf39ae7
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7
rename to sql/hive/src/test/resources/golden/timestamp_1-16-d0291a9bd42054b2732cb4f54cf39ae7
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-16-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-17-e7b398d2a8107a42419c83771bda41e6
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-16-e7b398d2a8107a42419c83771bda41e6
rename to sql/hive/src/test/resources/golden/timestamp_1-17-e7b398d2a8107a42419c83771bda41e6
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-18-a3eeec08bccae78d0d94ad2cb923e1cf
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf
rename to sql/hive/src/test/resources/golden/timestamp_1-18-a3eeec08bccae78d0d94ad2cb923e1cf
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-27-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-19-67f274bf16de625cf4e85af0c6185cac
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-27-67f274bf16de625cf4e85af0c6185cac
rename to sql/hive/src/test/resources/golden/timestamp_1-19-67f274bf16de625cf4e85af0c6185cac
diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d b/sql/hive/src/test/resources/golden/timestamp_1-2-1d7cf3a2512fa1876b422b79bbe05426
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d
rename to sql/hive/src/test/resources/golden/timestamp_1-2-1d7cf3a2512fa1876b422b79bbe05426
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 b/sql/hive/src/test/resources/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-19-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-20-343c75daac6695917608c17db8bf473e
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-19-343c75daac6695917608c17db8bf473e
rename to sql/hive/src/test/resources/golden/timestamp_1-20-343c75daac6695917608c17db8bf473e
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-21-cf19f7359a6d3456c4526b2c69f92d6a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a
rename to sql/hive/src/test/resources/golden/timestamp_1-21-cf19f7359a6d3456c4526b2c69f92d6a
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-21-d8fff1a6c464e50eb955babfafb0b98e b/sql/hive/src/test/resources/golden/timestamp_1-21-d8fff1a6c464e50eb955babfafb0b98e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 b/sql/hive/src/test/resources/golden/timestamp_1-22-cdb04b49b836e0244f6f0857aea7da8a
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9
rename to sql/hive/src/test/resources/golden/timestamp_1-22-cdb04b49b836e0244f6f0857aea7da8a
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-23-90269c1e50c7ae8e75ca9cc297982135
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135
rename to sql/hive/src/test/resources/golden/timestamp_1-23-90269c1e50c7ae8e75ca9cc297982135
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-24-e6bfca320c4ee3aff39cf2f179d57da6
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6
rename to sql/hive/src/test/resources/golden/timestamp_1-24-e6bfca320c4ee3aff39cf2f179d57da6
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-25-d0291a9bd42054b2732cb4f54cf39ae7
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7
rename to sql/hive/src/test/resources/golden/timestamp_1-25-d0291a9bd42054b2732cb4f54cf39ae7
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-25-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-26-e7b398d2a8107a42419c83771bda41e6
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-25-e7b398d2a8107a42419c83771bda41e6
rename to sql/hive/src/test/resources/golden/timestamp_1-26-e7b398d2a8107a42419c83771bda41e6
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-27-a3eeec08bccae78d0d94ad2cb923e1cf
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf
rename to sql/hive/src/test/resources/golden/timestamp_1-27-a3eeec08bccae78d0d94ad2cb923e1cf
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-36-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-28-67f274bf16de625cf4e85af0c6185cac
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-36-67f274bf16de625cf4e85af0c6185cac
rename to sql/hive/src/test/resources/golden/timestamp_1-28-67f274bf16de625cf4e85af0c6185cac
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-28-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-29-343c75daac6695917608c17db8bf473e
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-28-343c75daac6695917608c17db8bf473e
rename to sql/hive/src/test/resources/golden/timestamp_1-29-343c75daac6695917608c17db8bf473e
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c b/sql/hive/src/test/resources/golden/timestamp_1-3-74f477a8b726f5193dd42ef378a793c4
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c
rename to sql/hive/src/test/resources/golden/timestamp_1-3-74f477a8b726f5193dd42ef378a793c4
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 b/sql/hive/src/test/resources/golden/timestamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-30-273256141c33eb88194cad22eb940d21 b/sql/hive/src/test/resources/golden/timestamp_1-30-273256141c33eb88194cad22eb940d21
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-30-cf19f7359a6d3456c4526b2c69f92d6a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a
rename to sql/hive/src/test/resources/golden/timestamp_1-30-cf19f7359a6d3456c4526b2c69f92d6a
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 b/sql/hive/src/test/resources/golden/timestamp_1-31-9587976bd7e6caa5b667975c14e8dd53
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7
rename to sql/hive/src/test/resources/golden/timestamp_1-31-9587976bd7e6caa5b667975c14e8dd53
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-32-90269c1e50c7ae8e75ca9cc297982135
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135
rename to sql/hive/src/test/resources/golden/timestamp_1-32-90269c1e50c7ae8e75ca9cc297982135
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-33-e6bfca320c4ee3aff39cf2f179d57da6
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6
rename to sql/hive/src/test/resources/golden/timestamp_1-33-e6bfca320c4ee3aff39cf2f179d57da6
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-34-d0291a9bd42054b2732cb4f54cf39ae7
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7
rename to sql/hive/src/test/resources/golden/timestamp_1-34-d0291a9bd42054b2732cb4f54cf39ae7
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-34-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-35-e7b398d2a8107a42419c83771bda41e6
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-34-e7b398d2a8107a42419c83771bda41e6
rename to sql/hive/src/test/resources/golden/timestamp_1-35-e7b398d2a8107a42419c83771bda41e6
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-36-a3eeec08bccae78d0d94ad2cb923e1cf
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf
rename to sql/hive/src/test/resources/golden/timestamp_1-36-a3eeec08bccae78d0d94ad2cb923e1cf
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-45-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-37-67f274bf16de625cf4e85af0c6185cac
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-45-67f274bf16de625cf4e85af0c6185cac
rename to sql/hive/src/test/resources/golden/timestamp_1-37-67f274bf16de625cf4e85af0c6185cac
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-37-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-38-343c75daac6695917608c17db8bf473e
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-37-343c75daac6695917608c17db8bf473e
rename to sql/hive/src/test/resources/golden/timestamp_1-38-343c75daac6695917608c17db8bf473e
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d b/sql/hive/src/test/resources/golden/timestamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-39-cf19f7359a6d3456c4526b2c69f92d6a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a
rename to sql/hive/src/test/resources/golden/timestamp_1-39-cf19f7359a6d3456c4526b2c69f92d6a
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea b/sql/hive/src/test/resources/golden/timestamp_1-4-d833b177fac3162215468dde991f71d1
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea
rename to sql/hive/src/test/resources/golden/timestamp_1-4-d833b177fac3162215468dde991f71d1
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 b/sql/hive/src/test/resources/golden/timestamp_1-40-4ebcf4bcc059feba0fd9f76f26193f3b
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60
rename to sql/hive/src/test/resources/golden/timestamp_1-40-4ebcf4bcc059feba0fd9f76f26193f3b
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-13-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_1-41-90269c1e50c7ae8e75ca9cc297982135
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-13-25f6ec69328af6cba76899194e0dd84e
rename to sql/hive/src/test/resources/golden/timestamp_1-41-90269c1e50c7ae8e75ca9cc297982135
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-42-e6bfca320c4ee3aff39cf2f179d57da6
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6
rename to sql/hive/src/test/resources/golden/timestamp_1-42-e6bfca320c4ee3aff39cf2f179d57da6
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-43-d0291a9bd42054b2732cb4f54cf39ae7
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7
rename to sql/hive/src/test/resources/golden/timestamp_1-43-d0291a9bd42054b2732cb4f54cf39ae7
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-43-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-44-e7b398d2a8107a42419c83771bda41e6
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-43-e7b398d2a8107a42419c83771bda41e6
rename to sql/hive/src/test/resources/golden/timestamp_1-44-e7b398d2a8107a42419c83771bda41e6
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-45-a3eeec08bccae78d0d94ad2cb923e1cf
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf
rename to sql/hive/src/test/resources/golden/timestamp_1-45-a3eeec08bccae78d0d94ad2cb923e1cf
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-54-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-46-67f274bf16de625cf4e85af0c6185cac
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-54-67f274bf16de625cf4e85af0c6185cac
rename to sql/hive/src/test/resources/golden/timestamp_1-46-67f274bf16de625cf4e85af0c6185cac
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-46-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-47-343c75daac6695917608c17db8bf473e
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-46-343c75daac6695917608c17db8bf473e
rename to sql/hive/src/test/resources/golden/timestamp_1-47-343c75daac6695917608c17db8bf473e
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-48-7029255241de8e8b9710801319990044 b/sql/hive/src/test/resources/golden/timestamp_1-48-7029255241de8e8b9710801319990044
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-48-cf19f7359a6d3456c4526b2c69f92d6a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a
rename to sql/hive/src/test/resources/golden/timestamp_1-48-cf19f7359a6d3456c4526b2c69f92d6a
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 b/sql/hive/src/test/resources/golden/timestamp_1-49-7a59f9f939efc4b96f8159d00b39ed3
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64
rename to sql/hive/src/test/resources/golden/timestamp_1-49-7a59f9f939efc4b96f8159d00b39ed3
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-22-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_1-5-90269c1e50c7ae8e75ca9cc297982135
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-22-25f6ec69328af6cba76899194e0dd84e
rename to sql/hive/src/test/resources/golden/timestamp_1-5-90269c1e50c7ae8e75ca9cc297982135
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-31-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_1-50-90269c1e50c7ae8e75ca9cc297982135
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-31-25f6ec69328af6cba76899194e0dd84e
rename to sql/hive/src/test/resources/golden/timestamp_1-50-90269c1e50c7ae8e75ca9cc297982135
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-51-e6bfca320c4ee3aff39cf2f179d57da6
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6
rename to sql/hive/src/test/resources/golden/timestamp_1-51-e6bfca320c4ee3aff39cf2f179d57da6
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-52-d0291a9bd42054b2732cb4f54cf39ae7
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7
rename to sql/hive/src/test/resources/golden/timestamp_1-52-d0291a9bd42054b2732cb4f54cf39ae7
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-52-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-53-e7b398d2a8107a42419c83771bda41e6
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-52-e7b398d2a8107a42419c83771bda41e6
rename to sql/hive/src/test/resources/golden/timestamp_1-53-e7b398d2a8107a42419c83771bda41e6
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-54-a3eeec08bccae78d0d94ad2cb923e1cf
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf
rename to sql/hive/src/test/resources/golden/timestamp_1-54-a3eeec08bccae78d0d94ad2cb923e1cf
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-9-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-55-67f274bf16de625cf4e85af0c6185cac
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-9-67f274bf16de625cf4e85af0c6185cac
rename to sql/hive/src/test/resources/golden/timestamp_1-55-67f274bf16de625cf4e85af0c6185cac
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-55-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-56-343c75daac6695917608c17db8bf473e
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-55-343c75daac6695917608c17db8bf473e
rename to sql/hive/src/test/resources/golden/timestamp_1-56-343c75daac6695917608c17db8bf473e
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-57-cf19f7359a6d3456c4526b2c69f92d6a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a
rename to sql/hive/src/test/resources/golden/timestamp_1-57-cf19f7359a6d3456c4526b2c69f92d6a
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-57-d362501d0176855077e65f8faf067fa8 b/sql/hive/src/test/resources/golden/timestamp_1-57-d362501d0176855077e65f8faf067fa8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 b/sql/hive/src/test/resources/golden/timestamp_1-58-d362501d0176855077e65f8faf067fa8
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6
rename to sql/hive/src/test/resources/golden/timestamp_1-58-d362501d0176855077e65f8faf067fa8
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-6-e6bfca320c4ee3aff39cf2f179d57da6
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6
rename to sql/hive/src/test/resources/golden/timestamp_1-6-e6bfca320c4ee3aff39cf2f179d57da6
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-7-d0291a9bd42054b2732cb4f54cf39ae7
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7
rename to sql/hive/src/test/resources/golden/timestamp_1-7-d0291a9bd42054b2732cb4f54cf39ae7
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-7-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-8-e7b398d2a8107a42419c83771bda41e6
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-7-e7b398d2a8107a42419c83771bda41e6
rename to sql/hive/src/test/resources/golden/timestamp_1-8-e7b398d2a8107a42419c83771bda41e6
diff --git a/sql/hive/src/test/resources/golden/timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-9-a3eeec08bccae78d0d94ad2cb923e1cf
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf
rename to sql/hive/src/test/resources/golden/timestamp_1-9-a3eeec08bccae78d0d94ad2cb923e1cf
diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/timestamp_2-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/timestamp_2-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 b/sql/hive/src/test/resources/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 b/sql/hive/src/test/resources/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 b/sql/hive/src/test/resources/golden/timestamp_2-1-ea7192a4a5a985bcc8aab9aa79d9f028
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564
rename to sql/hive/src/test/resources/golden/timestamp_2-1-ea7192a4a5a985bcc8aab9aa79d9f028
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-18-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-10-252aebfe7882335d31bfc53a8705b7a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-18-252aebfe7882335d31bfc53a8705b7a
rename to sql/hive/src/test/resources/golden/timestamp_2-10-252aebfe7882335d31bfc53a8705b7a
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-10-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-11-5181279a0bf8939fe46ddacae015dad8
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-10-5181279a0bf8939fe46ddacae015dad8
rename to sql/hive/src/test/resources/golden/timestamp_2-11-5181279a0bf8939fe46ddacae015dad8
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-11-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-12-240fce5f58794fa051824e8732c00c03
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-11-240fce5f58794fa051824e8732c00c03
rename to sql/hive/src/test/resources/golden/timestamp_2-12-240fce5f58794fa051824e8732c00c03
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-12-7350308cbf49d6ebd6599d3802750acd b/sql/hive/src/test/resources/golden/timestamp_2-12-7350308cbf49d6ebd6599d3802750acd
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 b/sql/hive/src/test/resources/golden/timestamp_2-13-5f450162886ccc79af149541527f5643
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06
rename to sql/hive/src/test/resources/golden/timestamp_2-13-5f450162886ccc79af149541527f5643
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-4-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-14-25f6ec69328af6cba76899194e0dd84e
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-4-25f6ec69328af6cba76899194e0dd84e
rename to sql/hive/src/test/resources/golden/timestamp_2-14-25f6ec69328af6cba76899194e0dd84e
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-15-93c769be4cff93bea6e62bfe4e2a8742
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742
rename to sql/hive/src/test/resources/golden/timestamp_2-15-93c769be4cff93bea6e62bfe4e2a8742
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-16-5bdbf67419cc060b82d091d80ce59bf9
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9
rename to sql/hive/src/test/resources/golden/timestamp_2-16-5bdbf67419cc060b82d091d80ce59bf9
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-17-de3c42ab06c17ae895fd7deaf7bd9571
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571
rename to sql/hive/src/test/resources/golden/timestamp_2-17-de3c42ab06c17ae895fd7deaf7bd9571
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-18-da3937d21b7c2cfe1e624e812ae1d3ef
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef
rename to sql/hive/src/test/resources/golden/timestamp_2-18-da3937d21b7c2cfe1e624e812ae1d3ef
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-27-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-19-252aebfe7882335d31bfc53a8705b7a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-27-252aebfe7882335d31bfc53a8705b7a
rename to sql/hive/src/test/resources/golden/timestamp_2-19-252aebfe7882335d31bfc53a8705b7a
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 b/sql/hive/src/test/resources/golden/timestamp_2-2-61dbdf6d26c2a3f1143f6fdae999b1b4
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907
rename to sql/hive/src/test/resources/golden/timestamp_2-2-61dbdf6d26c2a3f1143f6fdae999b1b4
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 b/sql/hive/src/test/resources/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-19-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-20-5181279a0bf8939fe46ddacae015dad8
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-19-5181279a0bf8939fe46ddacae015dad8
rename to sql/hive/src/test/resources/golden/timestamp_2-20-5181279a0bf8939fe46ddacae015dad8
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-20-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-21-240fce5f58794fa051824e8732c00c03
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-20-240fce5f58794fa051824e8732c00c03
rename to sql/hive/src/test/resources/golden/timestamp_2-21-240fce5f58794fa051824e8732c00c03
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b b/sql/hive/src/test/resources/golden/timestamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 b/sql/hive/src/test/resources/golden/timestamp_2-22-469fe94fb60f4b00809190c303434641
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21
rename to sql/hive/src/test/resources/golden/timestamp_2-22-469fe94fb60f4b00809190c303434641
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-40-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-23-25f6ec69328af6cba76899194e0dd84e
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-40-25f6ec69328af6cba76899194e0dd84e
rename to sql/hive/src/test/resources/golden/timestamp_2-23-25f6ec69328af6cba76899194e0dd84e
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-24-93c769be4cff93bea6e62bfe4e2a8742
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742
rename to sql/hive/src/test/resources/golden/timestamp_2-24-93c769be4cff93bea6e62bfe4e2a8742
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-25-5bdbf67419cc060b82d091d80ce59bf9
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9
rename to sql/hive/src/test/resources/golden/timestamp_2-25-5bdbf67419cc060b82d091d80ce59bf9
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-26-de3c42ab06c17ae895fd7deaf7bd9571
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571
rename to sql/hive/src/test/resources/golden/timestamp_2-26-de3c42ab06c17ae895fd7deaf7bd9571
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-27-da3937d21b7c2cfe1e624e812ae1d3ef
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef
rename to sql/hive/src/test/resources/golden/timestamp_2-27-da3937d21b7c2cfe1e624e812ae1d3ef
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-36-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-28-252aebfe7882335d31bfc53a8705b7a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-36-252aebfe7882335d31bfc53a8705b7a
rename to sql/hive/src/test/resources/golden/timestamp_2-28-252aebfe7882335d31bfc53a8705b7a
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-28-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-29-5181279a0bf8939fe46ddacae015dad8
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-28-5181279a0bf8939fe46ddacae015dad8
rename to sql/hive/src/test/resources/golden/timestamp_2-29-5181279a0bf8939fe46ddacae015dad8
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 b/sql/hive/src/test/resources/golden/timestamp_2-3-a5092ff0f5a3d3b8f4171994932d4d19
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981
rename to sql/hive/src/test/resources/golden/timestamp_2-3-a5092ff0f5a3d3b8f4171994932d4d19
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 b/sql/hive/src/test/resources/golden/timestamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-29-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-30-240fce5f58794fa051824e8732c00c03
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-29-240fce5f58794fa051824e8732c00c03
rename to sql/hive/src/test/resources/golden/timestamp_2-30-240fce5f58794fa051824e8732c00c03
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-30-ffe6b6ddaaba84152074f7781fba2243 b/sql/hive/src/test/resources/golden/timestamp_2-30-ffe6b6ddaaba84152074f7781fba2243
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f b/sql/hive/src/test/resources/golden/timestamp_2-31-8f506498acf0c99c30960a00981ef460
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f
rename to sql/hive/src/test/resources/golden/timestamp_2-31-8f506498acf0c99c30960a00981ef460
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-49-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-32-25f6ec69328af6cba76899194e0dd84e
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-49-25f6ec69328af6cba76899194e0dd84e
rename to sql/hive/src/test/resources/golden/timestamp_2-32-25f6ec69328af6cba76899194e0dd84e
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-33-93c769be4cff93bea6e62bfe4e2a8742
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742
rename to sql/hive/src/test/resources/golden/timestamp_2-33-93c769be4cff93bea6e62bfe4e2a8742
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-34-5bdbf67419cc060b82d091d80ce59bf9
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9
rename to sql/hive/src/test/resources/golden/timestamp_2-34-5bdbf67419cc060b82d091d80ce59bf9
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-35-de3c42ab06c17ae895fd7deaf7bd9571
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571
rename to sql/hive/src/test/resources/golden/timestamp_2-35-de3c42ab06c17ae895fd7deaf7bd9571
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-36-da3937d21b7c2cfe1e624e812ae1d3ef
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef
rename to sql/hive/src/test/resources/golden/timestamp_2-36-da3937d21b7c2cfe1e624e812ae1d3ef
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-45-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-37-252aebfe7882335d31bfc53a8705b7a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-45-252aebfe7882335d31bfc53a8705b7a
rename to sql/hive/src/test/resources/golden/timestamp_2-37-252aebfe7882335d31bfc53a8705b7a
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-37-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-38-5181279a0bf8939fe46ddacae015dad8
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-37-5181279a0bf8939fe46ddacae015dad8
rename to sql/hive/src/test/resources/golden/timestamp_2-38-5181279a0bf8939fe46ddacae015dad8
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-38-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-39-240fce5f58794fa051824e8732c00c03
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-38-240fce5f58794fa051824e8732c00c03
rename to sql/hive/src/test/resources/golden/timestamp_2-39-240fce5f58794fa051824e8732c00c03
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-39-8236608f28681eac5503195096a34181 b/sql/hive/src/test/resources/golden/timestamp_2-39-8236608f28681eac5503195096a34181
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 b/sql/hive/src/test/resources/golden/timestamp_2-4-81d6d29dcb3fd12a519426dff64411d2
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7
rename to sql/hive/src/test/resources/golden/timestamp_2-4-81d6d29dcb3fd12a519426dff64411d2
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 b/sql/hive/src/test/resources/golden/timestamp_2-40-972a007e54d1c09e9ac9549c19a32dbb
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387
rename to sql/hive/src/test/resources/golden/timestamp_2-40-972a007e54d1c09e9ac9549c19a32dbb
diff --git a/sql/hive/src/test/resources/golden/timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4 b/sql/hive/src/test/resources/golden/timestamp_2-41-25f6ec69328af6cba76899194e0dd84e
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4
rename to sql/hive/src/test/resources/golden/timestamp_2-41-25f6ec69328af6cba76899194e0dd84e
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-42-93c769be4cff93bea6e62bfe4e2a8742
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742
rename to sql/hive/src/test/resources/golden/timestamp_2-42-93c769be4cff93bea6e62bfe4e2a8742
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-43-5bdbf67419cc060b82d091d80ce59bf9
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9
rename to sql/hive/src/test/resources/golden/timestamp_2-43-5bdbf67419cc060b82d091d80ce59bf9
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-44-de3c42ab06c17ae895fd7deaf7bd9571
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571
rename to sql/hive/src/test/resources/golden/timestamp_2-44-de3c42ab06c17ae895fd7deaf7bd9571
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-45-da3937d21b7c2cfe1e624e812ae1d3ef
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef
rename to sql/hive/src/test/resources/golden/timestamp_2-45-da3937d21b7c2cfe1e624e812ae1d3ef
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-54-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-46-252aebfe7882335d31bfc53a8705b7a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-54-252aebfe7882335d31bfc53a8705b7a
rename to sql/hive/src/test/resources/golden/timestamp_2-46-252aebfe7882335d31bfc53a8705b7a
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-46-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-47-5181279a0bf8939fe46ddacae015dad8
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-46-5181279a0bf8939fe46ddacae015dad8
rename to sql/hive/src/test/resources/golden/timestamp_2-47-5181279a0bf8939fe46ddacae015dad8
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-47-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-48-240fce5f58794fa051824e8732c00c03
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-47-240fce5f58794fa051824e8732c00c03
rename to sql/hive/src/test/resources/golden/timestamp_2-48-240fce5f58794fa051824e8732c00c03
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-48-654e5533ec6dc911996abc7e47af8ccb b/sql/hive/src/test/resources/golden/timestamp_2-48-654e5533ec6dc911996abc7e47af8ccb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f b/sql/hive/src/test/resources/golden/timestamp_2-49-650d2727b007638e0ed39b37c9498d66
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f
rename to sql/hive/src/test/resources/golden/timestamp_2-49-650d2727b007638e0ed39b37c9498d66
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4 b/sql/hive/src/test/resources/golden/timestamp_2-5-25f6ec69328af6cba76899194e0dd84e
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4
rename to sql/hive/src/test/resources/golden/timestamp_2-5-25f6ec69328af6cba76899194e0dd84e
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399 b/sql/hive/src/test/resources/golden/timestamp_2-50-25f6ec69328af6cba76899194e0dd84e
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399
rename to sql/hive/src/test/resources/golden/timestamp_2-50-25f6ec69328af6cba76899194e0dd84e
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-51-93c769be4cff93bea6e62bfe4e2a8742
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742
rename to sql/hive/src/test/resources/golden/timestamp_2-51-93c769be4cff93bea6e62bfe4e2a8742
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-52-5bdbf67419cc060b82d091d80ce59bf9
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9
rename to sql/hive/src/test/resources/golden/timestamp_2-52-5bdbf67419cc060b82d091d80ce59bf9
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-53-de3c42ab06c17ae895fd7deaf7bd9571
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571
rename to sql/hive/src/test/resources/golden/timestamp_2-53-de3c42ab06c17ae895fd7deaf7bd9571
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-54-da3937d21b7c2cfe1e624e812ae1d3ef
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef
rename to sql/hive/src/test/resources/golden/timestamp_2-54-da3937d21b7c2cfe1e624e812ae1d3ef
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-9-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-55-252aebfe7882335d31bfc53a8705b7a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-9-252aebfe7882335d31bfc53a8705b7a
rename to sql/hive/src/test/resources/golden/timestamp_2-55-252aebfe7882335d31bfc53a8705b7a
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-55-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-56-5181279a0bf8939fe46ddacae015dad8
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-55-5181279a0bf8939fe46ddacae015dad8
rename to sql/hive/src/test/resources/golden/timestamp_2-56-5181279a0bf8939fe46ddacae015dad8
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-56-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-57-240fce5f58794fa051824e8732c00c03
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-56-240fce5f58794fa051824e8732c00c03
rename to sql/hive/src/test/resources/golden/timestamp_2-57-240fce5f58794fa051824e8732c00c03
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 b/sql/hive/src/test/resources/golden/timestamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef b/sql/hive/src/test/resources/golden/timestamp_2-58-ea7192a4a5a985bcc8aab9aa79d9f028
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef
rename to sql/hive/src/test/resources/golden/timestamp_2-58-ea7192a4a5a985bcc8aab9aa79d9f028
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-6-93c769be4cff93bea6e62bfe4e2a8742
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742
rename to sql/hive/src/test/resources/golden/timestamp_2-6-93c769be4cff93bea6e62bfe4e2a8742
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-7-5bdbf67419cc060b82d091d80ce59bf9
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9
rename to sql/hive/src/test/resources/golden/timestamp_2-7-5bdbf67419cc060b82d091d80ce59bf9
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-8-de3c42ab06c17ae895fd7deaf7bd9571
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571
rename to sql/hive/src/test/resources/golden/timestamp_2-8-de3c42ab06c17ae895fd7deaf7bd9571
diff --git a/sql/hive/src/test/resources/golden/timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-9-da3937d21b7c2cfe1e624e812ae1d3ef
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef
rename to sql/hive/src/test/resources/golden/timestamp_2-9-da3937d21b7c2cfe1e624e812ae1d3ef
diff --git a/sql/hive/src/test/resources/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 b/sql/hive/src/test/resources/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/timestamp_3-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/timestamp_3-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 b/sql/hive/src/test/resources/golden/timestamp_3-1-165256158e3db1ce19c3c9db3c8011d2
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968
rename to sql/hive/src/test/resources/golden/timestamp_3-1-165256158e3db1ce19c3c9db3c8011d2
diff --git a/sql/hive/src/test/resources/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b b/sql/hive/src/test/resources/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_3-9-ffc79abb874323e165963aa39f460a9b b/sql/hive/src/test/resources/golden/timestamp_3-10-ffc79abb874323e165963aa39f460a9b
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_3-9-ffc79abb874323e165963aa39f460a9b
rename to sql/hive/src/test/resources/golden/timestamp_3-10-ffc79abb874323e165963aa39f460a9b
diff --git a/sql/hive/src/test/resources/golden/timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67 b/sql/hive/src/test/resources/golden/timestamp_3-11-7b1ec929239ee305ea9da46ebb990c67
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67
rename to sql/hive/src/test/resources/golden/timestamp_3-11-7b1ec929239ee305ea9da46ebb990c67
diff --git a/sql/hive/src/test/resources/golden/timestamp_3-12-165256158e3db1ce19c3c9db3c8011d2 b/sql/hive/src/test/resources/golden/timestamp_3-12-165256158e3db1ce19c3c9db3c8011d2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a b/sql/hive/src/test/resources/golden/timestamp_3-12-a63f40f6c4a022c16f8cf810e3b7ed2a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a
rename to sql/hive/src/test/resources/golden/timestamp_3-12-a63f40f6c4a022c16f8cf810e3b7ed2a
diff --git a/sql/hive/src/test/resources/golden/timestamp_3-13-7d225bcfa35f20da7dd63e7f8a413a77 b/sql/hive/src/test/resources/golden/timestamp_3-13-7d225bcfa35f20da7dd63e7f8a413a77
new file mode 100644
index 0000000000000000000000000000000000000000..4cfaa1b324da3e3087a092ce17a8292dc1663734
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/timestamp_3-13-7d225bcfa35f20da7dd63e7f8a413a77
@@ -0,0 +1 @@
+2011-04-29 20:46:56.4485	1.3041352164485E9	1	1.3041352164485E9	1.3041352164485E9
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 b/sql/hive/src/test/resources/golden/timestamp_3-14-165256158e3db1ce19c3c9db3c8011d2
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610
rename to sql/hive/src/test/resources/golden/timestamp_3-14-165256158e3db1ce19c3c9db3c8011d2
diff --git a/sql/hive/src/test/resources/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 b/sql/hive/src/test/resources/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d b/sql/hive/src/test/resources/golden/timestamp_3-2-81edf5107270547641586aa02b4e7d9b
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d
rename to sql/hive/src/test/resources/golden/timestamp_3-2-81edf5107270547641586aa02b4e7d9b
diff --git a/sql/hive/src/test/resources/golden/timestamp_3-3-6143888a940bfcac1133330764f5a31a b/sql/hive/src/test/resources/golden/timestamp_3-3-6143888a940bfcac1133330764f5a31a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b b/sql/hive/src/test/resources/golden/timestamp_3-3-7a012a0d98729da25b5ac374855dcee4
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b
rename to sql/hive/src/test/resources/golden/timestamp_3-3-7a012a0d98729da25b5ac374855dcee4
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad b/sql/hive/src/test/resources/golden/timestamp_3-4-86514381187b246a5685577c1968c559
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad
rename to sql/hive/src/test/resources/golden/timestamp_3-4-86514381187b246a5685577c1968c559
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453 b/sql/hive/src/test/resources/golden/timestamp_3-5-935d0d2492beab99bbbba26ba62a1db4
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453
rename to sql/hive/src/test/resources/golden/timestamp_3-5-935d0d2492beab99bbbba26ba62a1db4
diff --git a/sql/hive/src/test/resources/golden/timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 b/sql/hive/src/test/resources/golden/timestamp_3-6-8fe348d5d9b9903a26eda32d308b8e41
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41
rename to sql/hive/src/test/resources/golden/timestamp_3-6-8fe348d5d9b9903a26eda32d308b8e41
diff --git a/sql/hive/src/test/resources/golden/timestamp_3-6-6be5fe01c502cd24db32a3781c97a703 b/sql/hive/src/test/resources/golden/timestamp_3-7-6be5fe01c502cd24db32a3781c97a703
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_3-6-6be5fe01c502cd24db32a3781c97a703
rename to sql/hive/src/test/resources/golden/timestamp_3-7-6be5fe01c502cd24db32a3781c97a703
diff --git a/sql/hive/src/test/resources/golden/timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add b/sql/hive/src/test/resources/golden/timestamp_3-8-6066ba0451cd0fcfac4bea6376e72add
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add
rename to sql/hive/src/test/resources/golden/timestamp_3-8-6066ba0451cd0fcfac4bea6376e72add
diff --git a/sql/hive/src/test/resources/golden/timestamp_3-8-22e03daa775eab145d39ec0730953f7e b/sql/hive/src/test/resources/golden/timestamp_3-9-22e03daa775eab145d39ec0730953f7e
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_3-8-22e03daa775eab145d39ec0730953f7e
rename to sql/hive/src/test/resources/golden/timestamp_3-9-22e03daa775eab145d39ec0730953f7e
diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/timestamp_comparison-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/timestamp_comparison-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870 b/sql/hive/src/test/resources/golden/timestamp_comparison-1-4b68f7ad0f8cf337e42bf16a45e15818
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870
rename to sql/hive/src/test/resources/golden/timestamp_comparison-1-4b68f7ad0f8cf337e42bf16a45e15818
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d b/sql/hive/src/test/resources/golden/timestamp_comparison-2-60557e7bd2822c89fa8b076a9d0520fc
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d
rename to sql/hive/src/test/resources/golden/timestamp_comparison-2-60557e7bd2822c89fa8b076a9d0520fc
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc b/sql/hive/src/test/resources/golden/timestamp_comparison-3-f96a9d88327951bd93f672dc2463ecd4
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc
rename to sql/hive/src/test/resources/golden/timestamp_comparison-3-f96a9d88327951bd93f672dc2463ecd4
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-14-773801b833cf72d35016916b786275b5 b/sql/hive/src/test/resources/golden/timestamp_comparison-4-13e17ed811165196416f777cbc162592
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-14-773801b833cf72d35016916b786275b5
rename to sql/hive/src/test/resources/golden/timestamp_comparison-4-13e17ed811165196416f777cbc162592
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65 b/sql/hive/src/test/resources/golden/timestamp_comparison-5-4fa8a36edbefde4427c2ab2cf30e6399
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65
rename to sql/hive/src/test/resources/golden/timestamp_comparison-5-4fa8a36edbefde4427c2ab2cf30e6399
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 b/sql/hive/src/test/resources/golden/timestamp_comparison-6-7e4fb6e8ba01df422e4c67e06a0c8453
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670
rename to sql/hive/src/test/resources/golden/timestamp_comparison-6-7e4fb6e8ba01df422e4c67e06a0c8453
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0 b/sql/hive/src/test/resources/golden/timestamp_comparison-7-8c8e73673a950f6b3d960b08fcea076f
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0
rename to sql/hive/src/test/resources/golden/timestamp_comparison-7-8c8e73673a950f6b3d960b08fcea076f
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 b/sql/hive/src/test/resources/golden/timestamp_comparison-8-510c0a2a57dc5df8588bd13c4152f8bc
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9
rename to sql/hive/src/test/resources/golden/timestamp_comparison-8-510c0a2a57dc5df8588bd13c4152f8bc
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 b/sql/hive/src/test/resources/golden/timestamp_comparison-9-659d5b1ae8200f13f265270e52a3dd65
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845
rename to sql/hive/src/test/resources/golden/timestamp_comparison-9-659d5b1ae8200f13f265270e52a3dd65
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 b/sql/hive/src/test/resources/golden/timestamp_lazy-2-bb5a4a13274290029bd07d95c2f92563
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67
rename to sql/hive/src/test/resources/golden/timestamp_lazy-2-bb5a4a13274290029bd07d95c2f92563
diff --git a/sql/hive/src/test/resources/golden/timestamp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 b/sql/hive/src/test/resources/golden/timestamp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c b/sql/hive/src/test/resources/golden/timestamp_null-2-51762cf5079877abf7d81127738f4e5
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c
rename to sql/hive/src/test/resources/golden/timestamp_null-2-51762cf5079877abf7d81127738f4e5
diff --git a/sql/hive/src/test/resources/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 b/sql/hive/src/test/resources/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/timestamp_udf-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/timestamp_udf-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af b/sql/hive/src/test/resources/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d b/sql/hive/src/test/resources/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a b/sql/hive/src/test/resources/golden/timestamp_udf-1-79914c5347620c6e62a8e0b9a95984af
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a
rename to sql/hive/src/test/resources/golden/timestamp_udf-1-79914c5347620c6e62a8e0b9a95984af
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-18-cb033ecad964a2623bc633ac1d3f752a b/sql/hive/src/test/resources/golden/timestamp_udf-10-287614364eaa3fb82aad08c6b62cc938
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_udf-18-cb033ecad964a2623bc633ac1d3f752a
rename to sql/hive/src/test/resources/golden/timestamp_udf-10-287614364eaa3fb82aad08c6b62cc938
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-10-dbc23736a61d9482d13cacada02a7a09 b/sql/hive/src/test/resources/golden/timestamp_udf-11-dbc23736a61d9482d13cacada02a7a09
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_udf-10-dbc23736a61d9482d13cacada02a7a09
rename to sql/hive/src/test/resources/golden/timestamp_udf-11-dbc23736a61d9482d13cacada02a7a09
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-11-442cf850a0cc1f1dcfdeaeffbffb2c35 b/sql/hive/src/test/resources/golden/timestamp_udf-12-442cf850a0cc1f1dcfdeaeffbffb2c35
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_udf-11-442cf850a0cc1f1dcfdeaeffbffb2c35
rename to sql/hive/src/test/resources/golden/timestamp_udf-12-442cf850a0cc1f1dcfdeaeffbffb2c35
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-12-51959036fd4ac4f1e24f4e06eb9b0b6 b/sql/hive/src/test/resources/golden/timestamp_udf-13-51959036fd4ac4f1e24f4e06eb9b0b6
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_udf-12-51959036fd4ac4f1e24f4e06eb9b0b6
rename to sql/hive/src/test/resources/golden/timestamp_udf-13-51959036fd4ac4f1e24f4e06eb9b0b6
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-13-6ab3f356deaf807e8accc37e1f4849a b/sql/hive/src/test/resources/golden/timestamp_udf-14-6ab3f356deaf807e8accc37e1f4849a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_udf-13-6ab3f356deaf807e8accc37e1f4849a
rename to sql/hive/src/test/resources/golden/timestamp_udf-14-6ab3f356deaf807e8accc37e1f4849a
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-14-c745a1016461403526d44928a269c1de b/sql/hive/src/test/resources/golden/timestamp_udf-15-c745a1016461403526d44928a269c1de
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_udf-14-c745a1016461403526d44928a269c1de
rename to sql/hive/src/test/resources/golden/timestamp_udf-15-c745a1016461403526d44928a269c1de
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-15-7ab76c4458c7f78038c8b1df0fdeafbe b/sql/hive/src/test/resources/golden/timestamp_udf-16-7ab76c4458c7f78038c8b1df0fdeafbe
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_udf-15-7ab76c4458c7f78038c8b1df0fdeafbe
rename to sql/hive/src/test/resources/golden/timestamp_udf-16-7ab76c4458c7f78038c8b1df0fdeafbe
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-16-b36e87e17ca24d82072220bff559c718 b/sql/hive/src/test/resources/golden/timestamp_udf-17-b36e87e17ca24d82072220bff559c718
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_udf-16-b36e87e17ca24d82072220bff559c718
rename to sql/hive/src/test/resources/golden/timestamp_udf-17-b36e87e17ca24d82072220bff559c718
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-17-dad44d2d4a421286e9da080271bd2639 b/sql/hive/src/test/resources/golden/timestamp_udf-18-dad44d2d4a421286e9da080271bd2639
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_udf-17-dad44d2d4a421286e9da080271bd2639
rename to sql/hive/src/test/resources/golden/timestamp_udf-18-dad44d2d4a421286e9da080271bd2639
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-19-79914c5347620c6e62a8e0b9a95984af b/sql/hive/src/test/resources/golden/timestamp_udf-19-79914c5347620c6e62a8e0b9a95984af
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-9-287614364eaa3fb82aad08c6b62cc938 b/sql/hive/src/test/resources/golden/timestamp_udf-19-cb033ecad964a2623bc633ac1d3f752a
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_udf-9-287614364eaa3fb82aad08c6b62cc938
rename to sql/hive/src/test/resources/golden/timestamp_udf-19-cb033ecad964a2623bc633ac1d3f752a
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 b/sql/hive/src/test/resources/golden/timestamp_udf-2-59fc1842a23369235d42ed040d45fb3d
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648
rename to sql/hive/src/test/resources/golden/timestamp_udf-2-59fc1842a23369235d42ed040d45fb3d
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 b/sql/hive/src/test/resources/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-20-59fc1842a23369235d42ed040d45fb3d b/sql/hive/src/test/resources/golden/timestamp_udf-20-59fc1842a23369235d42ed040d45fb3d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 b/sql/hive/src/test/resources/golden/timestamp_udf-20-79914c5347620c6e62a8e0b9a95984af
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279
rename to sql/hive/src/test/resources/golden/timestamp_udf-20-79914c5347620c6e62a8e0b9a95984af
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa b/sql/hive/src/test/resources/golden/timestamp_udf-21-59fc1842a23369235d42ed040d45fb3d
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa
rename to sql/hive/src/test/resources/golden/timestamp_udf-21-59fc1842a23369235d42ed040d45fb3d
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 b/sql/hive/src/test/resources/golden/timestamp_udf-3-9039f474f9a96e9f15ace528faeed923
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2
rename to sql/hive/src/test/resources/golden/timestamp_udf-3-9039f474f9a96e9f15ace528faeed923
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f b/sql/hive/src/test/resources/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-4-80ce02ec84ee8abcb046367ca37279cc b/sql/hive/src/test/resources/golden/timestamp_udf-4-80ce02ec84ee8abcb046367ca37279cc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/timestamp_udf-4-b0fd4ca3b22eb732a32772399331352f
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b
rename to sql/hive/src/test/resources/golden/timestamp_udf-4-b0fd4ca3b22eb732a32772399331352f
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 b/sql/hive/src/test/resources/golden/timestamp_udf-5-66868a2b075de978784011e9955483d
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434
rename to sql/hive/src/test/resources/golden/timestamp_udf-5-66868a2b075de978784011e9955483d
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-5-1124399033bcadf3874fb48f593392d b/sql/hive/src/test/resources/golden/timestamp_udf-6-1124399033bcadf3874fb48f593392d
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_udf-5-1124399033bcadf3874fb48f593392d
rename to sql/hive/src/test/resources/golden/timestamp_udf-6-1124399033bcadf3874fb48f593392d
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-6-5810193ce35d38c23f4fc4b4979d60a4 b/sql/hive/src/test/resources/golden/timestamp_udf-7-5810193ce35d38c23f4fc4b4979d60a4
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_udf-6-5810193ce35d38c23f4fc4b4979d60a4
rename to sql/hive/src/test/resources/golden/timestamp_udf-7-5810193ce35d38c23f4fc4b4979d60a4
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-7-250e640a6a818f989f3f3280b00f64f9 b/sql/hive/src/test/resources/golden/timestamp_udf-8-250e640a6a818f989f3f3280b00f64f9
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_udf-7-250e640a6a818f989f3f3280b00f64f9
rename to sql/hive/src/test/resources/golden/timestamp_udf-8-250e640a6a818f989f3f3280b00f64f9
diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-8-975df43df015d86422965af456f87a94 b/sql/hive/src/test/resources/golden/timestamp_udf-9-975df43df015d86422965af456f87a94
similarity index 100%
rename from sql/hive/src/test/resources/golden/timestamp_udf-8-975df43df015d86422965af456f87a94
rename to sql/hive/src/test/resources/golden/timestamp_udf-9-975df43df015d86422965af456f87a94
diff --git a/sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe b/sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe
index d23e05acf7ba58c8471721a7bb93d585e57e75c8..e34118512c1d769075d0b492b217316096597c08 100644
--- a/sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe
+++ b/sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe
@@ -497,4 +497,4 @@
 403
 400
 200
-97
\ No newline at end of file
+97
diff --git a/sql/hive/src/test/resources/golden/transform1-0-b6919fc48901e388c869c84ae0211102 b/sql/hive/src/test/resources/golden/transform1-0-b6919fc48901e388c869c84ae0211102
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 b/sql/hive/src/test/resources/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 b/sql/hive/src/test/resources/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 b/sql/hive/src/test/resources/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/transform1-4-65527bae8e73262255ef83082c6968f9 b/sql/hive/src/test/resources/golden/transform1-4-65527bae8e73262255ef83082c6968f9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 b/sql/hive/src/test/resources/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f b/sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f
deleted file mode 100644
index c6f628b1a3eef684287f4f004254c4e3c1663049..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f
+++ /dev/null
@@ -1 +0,0 @@
-[0,1,2]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 b/sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47
index 839efe2d57956a151ce2b7965e3d144ae5e39b63..9370a501a5179ad0896c6b5e46d17a726cd57b61 100644
--- a/sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47
+++ b/sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47
@@ -165,4 +165,4 @@
 98	val_98
 98	val_98
 98	val_98
-98	val_98
\ No newline at end of file
+98	val_98
diff --git a/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161
+++ b/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 b/sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99
index 839efe2d57956a151ce2b7965e3d144ae5e39b63..9370a501a5179ad0896c6b5e46d17a726cd57b61 100644
--- a/sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99
+++ b/sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99
@@ -165,4 +165,4 @@
 98	val_98
 98	val_98
 98	val_98
-98	val_98
\ No newline at end of file
+98	val_98
diff --git a/sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d b/sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d
index 66fafbdf72b63bb93673817ae0e18ca07b2da193..3df4716f0b05f86e8801d5d86bf0d194c85857e6 100644
--- a/sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d	
+++ b/sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d	
@@ -1025,4 +1025,4 @@
 200	val_200	200	val_200
 200	val_200	200	val_200
 97	val_97	97	val_97
-97	val_97	97	val_97
\ No newline at end of file
+97	val_97	97	val_97
diff --git a/sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 b/sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3
index 66fafbdf72b63bb93673817ae0e18ca07b2da193..3df4716f0b05f86e8801d5d86bf0d194c85857e6 100644
--- a/sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3	
+++ b/sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3	
@@ -1025,4 +1025,4 @@
 200	val_200	200	val_200
 200	val_200	200	val_200
 97	val_97	97	val_97
-97	val_97	97	val_97
\ No newline at end of file
+97	val_97	97	val_97
diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/type_cast_1-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/type_cast_1-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/type_cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d b/sql/hive/src/test/resources/golden/type_cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 b/sql/hive/src/test/resources/golden/type_cast_1-1-60ea21e6e7d054a65f959fc89acf1b3d
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065
rename to sql/hive/src/test/resources/golden/type_cast_1-1-60ea21e6e7d054a65f959fc89acf1b3d
diff --git a/sql/hive/src/test/resources/golden/type_cast_1-2-53a667981ad567b2ab977f67d65c5825 b/sql/hive/src/test/resources/golden/type_cast_1-2-53a667981ad567b2ab977f67d65c5825
new file mode 100644
index 0000000000000000000000000000000000000000..7ed6ff82de6bcc2a78243fc9c54d3ef5ac14da69
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/type_cast_1-2-53a667981ad567b2ab977f67d65c5825
@@ -0,0 +1 @@
+5
diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/type_widening-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/type_widening-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 b/sql/hive/src/test/resources/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 b/sql/hive/src/test/resources/golden/type_widening-1-630ac2c7e7dea4837384ccd572209229
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2
rename to sql/hive/src/test/resources/golden/type_widening-1-630ac2c7e7dea4837384ccd572209229
diff --git a/sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 b/sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 b/sql/hive/src/test/resources/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/type_widening-2-cfbdf2b6fca84c6e23d4e691d2221bd6
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/type_widening-2-cfbdf2b6fca84c6e23d4e691d2221bd6
diff --git a/sql/hive/src/test/resources/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 b/sql/hive/src/test/resources/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04
deleted file mode 100644
index cf940f4c5faa8e36029a5e48c559b2bf161cd817..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04
+++ /dev/null
@@ -1,1000 +0,0 @@
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-0
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
-9223372036854775807
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c b/sql/hive/src/test/resources/golden/type_widening-3-a0ba6952d9bf830d1d1ea7aebd3784a2
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c
rename to sql/hive/src/test/resources/golden/type_widening-3-a0ba6952d9bf830d1d1ea7aebd3784a2
diff --git a/sql/hive/src/test/resources/golden/type_widening-4-65da8c67f6903286168acb39ac67fc04 b/sql/hive/src/test/resources/golden/type_widening-4-65da8c67f6903286168acb39ac67fc04
new file mode 100644
index 0000000000000000000000000000000000000000..00841d23b3f94220ada669a10a6c9849a9ee512f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/type_widening-4-65da8c67f6903286168acb39ac67fc04
@@ -0,0 +1,1000 @@
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+0
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
+9223372036854775807
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e b/sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e
index fed3d6802023ab4bb6f5d2f4aa5d895e45ec7524..bb3393324db7a36e78bc14eadbc26d64df6b3e49 100644
--- a/sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e
+++ b/sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e
@@ -1 +1 @@
-collect_set(x) - Returns a set of objects with duplicate elements eliminated
\ No newline at end of file
+collect_set(x) - Returns a set of objects with duplicate elements eliminated
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 b/sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449
index fed3d6802023ab4bb6f5d2f4aa5d895e45ec7524..bb3393324db7a36e78bc14eadbc26d64df6b3e49 100644
--- a/sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449
+++ b/sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449
@@ -1 +1 @@
-collect_set(x) - Returns a set of objects with duplicate elements eliminated
\ No newline at end of file
+collect_set(x) - Returns a set of objects with duplicate elements eliminated
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db
index cedc3068ee4a1db66560464e8c1cc0a4ffcc9a0f..c87ba74c9000a6e33bc91e7d663e173f67ca7bf7 100644
--- a/sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db
+++ b/sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db
@@ -17,4 +17,4 @@
 27	["val_27"]
 28	["val_28"]
 30	["val_30"]
-33	["val_33"]
\ No newline at end of file
+33	["val_33"]
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-11-5c3768074977ef68a1b9bb72eb9ef02 b/sql/hive/src/test/resources/golden/udaf_collect_set-11-5c3768074977ef68a1b9bb72eb9ef02
new file mode 100644
index 0000000000000000000000000000000000000000..337e96635cc70f26d836dbc0c951d9f0f984c38c
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udaf_collect_set-11-5c3768074977ef68a1b9bb72eb9ef02
@@ -0,0 +1,20 @@
+0	["val_0","val_0","val_0"]
+2	["val_2"]
+4	["val_4"]
+5	["val_5","val_5","val_5"]
+8	["val_8"]
+9	["val_9"]
+10	["val_10"]
+11	["val_11"]
+12	["val_12","val_12"]
+15	["val_15","val_15"]
+17	["val_17"]
+18	["val_18","val_18"]
+19	["val_19"]
+20	["val_20"]
+24	["val_24","val_24"]
+26	["val_26","val_26"]
+27	["val_27"]
+28	["val_28"]
+30	["val_30"]
+33	["val_33"]
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udaf_collect_set-12-1d351f7e821fcaf66c6f7503e42fb291
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/udaf_collect_set-12-1d351f7e821fcaf66c6f7503e42fb291
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udaf_collect_set-13-a00d1791b7fa7ac5a0505d95c3d12257
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/udaf_collect_set-13-a00d1791b7fa7ac5a0505d95c3d12257
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db
deleted file mode 100644
index cedc3068ee4a1db66560464e8c1cc0a4ffcc9a0f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db
+++ /dev/null
@@ -1,20 +0,0 @@
-0	["val_0"]
-2	["val_2"]
-4	["val_4"]
-5	["val_5"]
-8	["val_8"]
-9	["val_9"]
-10	["val_10"]
-11	["val_11"]
-12	["val_12"]
-15	["val_15"]
-17	["val_17"]
-18	["val_18"]
-19	["val_19"]
-20	["val_20"]
-24	["val_24"]
-26	["val_26"]
-27	["val_27"]
-28	["val_28"]
-30	["val_30"]
-33	["val_33"]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-14-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-14-c8bc33095e1a195bb7b5e579d8d78db
new file mode 100644
index 0000000000000000000000000000000000000000..c87ba74c9000a6e33bc91e7d663e173f67ca7bf7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udaf_collect_set-14-c8bc33095e1a195bb7b5e579d8d78db
@@ -0,0 +1,20 @@
+0	["val_0"]
+2	["val_2"]
+4	["val_4"]
+5	["val_5"]
+8	["val_8"]
+9	["val_9"]
+10	["val_10"]
+11	["val_11"]
+12	["val_12"]
+15	["val_15"]
+17	["val_17"]
+18	["val_18"]
+19	["val_19"]
+20	["val_20"]
+24	["val_24"]
+26	["val_26"]
+27	["val_27"]
+28	["val_28"]
+30	["val_30"]
+33	["val_33"]
diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udaf_collect_set-15-863233ccd616401efb4bf83c4b9e3a52
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/udaf_collect_set-15-863233ccd616401efb4bf83c4b9e3a52
diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udaf_collect_set-16-a00d1791b7fa7ac5a0505d95c3d12257
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/udaf_collect_set-16-a00d1791b7fa7ac5a0505d95c3d12257
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-17-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-17-c8bc33095e1a195bb7b5e579d8d78db
new file mode 100644
index 0000000000000000000000000000000000000000..c87ba74c9000a6e33bc91e7d663e173f67ca7bf7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udaf_collect_set-17-c8bc33095e1a195bb7b5e579d8d78db
@@ -0,0 +1,20 @@
+0	["val_0"]
+2	["val_2"]
+4	["val_4"]
+5	["val_5"]
+8	["val_8"]
+9	["val_9"]
+10	["val_10"]
+11	["val_11"]
+12	["val_12"]
+15	["val_15"]
+17	["val_17"]
+18	["val_18"]
+19	["val_19"]
+20	["val_20"]
+24	["val_24"]
+26	["val_26"]
+27	["val_27"]
+28	["val_28"]
+30	["val_30"]
+33	["val_33"]
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-2-4747c35670a011344982573ba31a9bb b/sql/hive/src/test/resources/golden/udaf_collect_set-2-4747c35670a011344982573ba31a9bb
new file mode 100644
index 0000000000000000000000000000000000000000..28abc06ee9140a98b494288c1212d432d35c9316
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udaf_collect_set-2-4747c35670a011344982573ba31a9bb
@@ -0,0 +1 @@
+collect_list(x) - Returns a list of objects with duplicates
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-3-9aa348a25ca17ab5b636d3ea2d6df986 b/sql/hive/src/test/resources/golden/udaf_collect_set-3-9aa348a25ca17ab5b636d3ea2d6df986
new file mode 100644
index 0000000000000000000000000000000000000000..28abc06ee9140a98b494288c1212d432d35c9316
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udaf_collect_set-3-9aa348a25ca17ab5b636d3ea2d6df986
@@ -0,0 +1 @@
+collect_list(x) - Returns a list of objects with duplicates
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udaf_collect_set-4-1d351f7e821fcaf66c6f7503e42fb291
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/udaf_collect_set-4-1d351f7e821fcaf66c6f7503e42fb291
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db
deleted file mode 100644
index cedc3068ee4a1db66560464e8c1cc0a4ffcc9a0f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db
+++ /dev/null
@@ -1,20 +0,0 @@
-0	["val_0"]
-2	["val_2"]
-4	["val_4"]
-5	["val_5"]
-8	["val_8"]
-9	["val_9"]
-10	["val_10"]
-11	["val_11"]
-12	["val_12"]
-15	["val_15"]
-17	["val_17"]
-18	["val_18"]
-19	["val_19"]
-20	["val_20"]
-24	["val_24"]
-26	["val_26"]
-27	["val_27"]
-28	["val_28"]
-30	["val_30"]
-33	["val_33"]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udaf_collect_set-5-a7dc16cb82c595b18d4258a38a304b1e
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/udaf_collect_set-5-a7dc16cb82c595b18d4258a38a304b1e
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-6-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-6-c8bc33095e1a195bb7b5e579d8d78db
new file mode 100644
index 0000000000000000000000000000000000000000..c87ba74c9000a6e33bc91e7d663e173f67ca7bf7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udaf_collect_set-6-c8bc33095e1a195bb7b5e579d8d78db
@@ -0,0 +1,20 @@
+0	["val_0"]
+2	["val_2"]
+4	["val_4"]
+5	["val_5"]
+8	["val_8"]
+9	["val_9"]
+10	["val_10"]
+11	["val_11"]
+12	["val_12"]
+15	["val_15"]
+17	["val_17"]
+18	["val_18"]
+19	["val_19"]
+20	["val_20"]
+24	["val_24"]
+26	["val_26"]
+27	["val_27"]
+28	["val_28"]
+30	["val_30"]
+33	["val_33"]
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-7-1fd4f3dcdac818ccc95c5033c6d01b56 b/sql/hive/src/test/resources/golden/udaf_collect_set-7-1fd4f3dcdac818ccc95c5033c6d01b56
new file mode 100644
index 0000000000000000000000000000000000000000..337e96635cc70f26d836dbc0c951d9f0f984c38c
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udaf_collect_set-7-1fd4f3dcdac818ccc95c5033c6d01b56
@@ -0,0 +1,20 @@
+0	["val_0","val_0","val_0"]
+2	["val_2"]
+4	["val_4"]
+5	["val_5","val_5","val_5"]
+8	["val_8"]
+9	["val_9"]
+10	["val_10"]
+11	["val_11"]
+12	["val_12","val_12"]
+15	["val_15","val_15"]
+17	["val_17"]
+18	["val_18","val_18"]
+19	["val_19"]
+20	["val_20"]
+24	["val_24","val_24"]
+26	["val_26","val_26"]
+27	["val_27"]
+28	["val_28"]
+30	["val_30"]
+33	["val_33"]
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db
deleted file mode 100644
index cedc3068ee4a1db66560464e8c1cc0a4ffcc9a0f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db
+++ /dev/null
@@ -1,20 +0,0 @@
-0	["val_0"]
-2	["val_2"]
-4	["val_4"]
-5	["val_5"]
-8	["val_8"]
-9	["val_9"]
-10	["val_10"]
-11	["val_11"]
-12	["val_12"]
-15	["val_15"]
-17	["val_17"]
-18	["val_18"]
-19	["val_19"]
-20	["val_20"]
-24	["val_24"]
-26	["val_26"]
-27	["val_27"]
-28	["val_28"]
-30	["val_30"]
-33	["val_33"]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udaf_collect_set-8-863233ccd616401efb4bf83c4b9e3a52
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/udaf_collect_set-8-863233ccd616401efb4bf83c4b9e3a52
diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udaf_collect_set-9-a7dc16cb82c595b18d4258a38a304b1e
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/udaf_collect_set-9-a7dc16cb82c595b18d4258a38a304b1e
diff --git a/sql/hive/src/test/resources/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 b/sql/hive/src/test/resources/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 b/sql/hive/src/test/resources/golden/udaf_corr-2-e886f45c8f085596ffd420f89cdc2909
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01
rename to sql/hive/src/test/resources/golden/udaf_corr-2-e886f45c8f085596ffd420f89cdc2909
diff --git a/sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e b/sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e
index 6e5d422b3132e61254099a0938dbd7f2f7eba299..06b8ee9e72496b116b12157d60145511df23f322 100644
--- a/sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e
+++ b/sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e
@@ -1,2 +1,2 @@
 corr(x,y) - Returns the Pearson coefficient of correlation
-between a set of number pairs
\ No newline at end of file
+between a set of number pairs
diff --git a/sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a b/sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a
index fcb49ae69f74f5355086a6653d5dacfe3ae0c026..f516ef3a30fdf4223c723bfa5a10ef47ec2d7021 100644
--- a/sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a
+++ b/sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a
@@ -6,4 +6,4 @@ a singleton set, NULL will be returned. Otherwise, it computes the following:
    COVAR_POP(x,y)/(STDDEV_POP(x)*STDDEV_POP(y))
 where neither x nor y is null,
 COVAR_POP is the population covariance,
-and STDDEV_POP is the population standard deviation.
\ No newline at end of file
+and STDDEV_POP is the population standard deviation.
diff --git a/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb b/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..7951defec192aa41c72f62ac9c9f4b001cdaaba8 100644
--- a/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb
+++ b/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb
@@ -1 +1 @@
-NULL
\ No newline at end of file
+NULL
diff --git a/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 b/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..7951defec192aa41c72f62ac9c9f4b001cdaaba8 100644
--- a/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727
+++ b/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727
@@ -1 +1 @@
-NULL
\ No newline at end of file
+NULL
diff --git a/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 b/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..7951defec192aa41c72f62ac9c9f4b001cdaaba8 100644
--- a/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3
+++ b/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3
@@ -1 +1 @@
-NULL
\ No newline at end of file
+NULL
diff --git a/sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 b/sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369
index 3f730875aef8cabf7e7495a3619fa347993d013b..946c9d58047e5fdddf578a9e483208de3b97a50a 100644
--- a/sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369
+++ b/sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369
@@ -3,4 +3,4 @@
 3	NULL
 4	NULL
 5	NULL
-6	NULL
\ No newline at end of file
+6	NULL
diff --git a/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 b/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16
index 5d97236e8b03fc500d6d3fd75876bfd086b87404..011d78d68766d0a96649ab2f6dcc1390d939ed8e 100644
--- a/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16
+++ b/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16
@@ -1 +1 @@
-0.6633880657639323
\ No newline at end of file
+0.6633880657639323
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 b/sql/hive/src/test/resources/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 b/sql/hive/src/test/resources/golden/udaf_covar_pop-2-e886f45c8f085596ffd420f89cdc2909
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0
rename to sql/hive/src/test/resources/golden/udaf_covar_pop-2-e886f45c8f085596ffd420f89cdc2909
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 b/sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6
index 0fed030c22af33ccbc55d09c530894e9ce881b7a..eadc2e1aa99b719272f941e44af80b0cdd13f273 100644
--- a/sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6
+++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6
@@ -1 +1 @@
-covar_pop(x,y) - Returns the population covariance of a set of number pairs
\ No newline at end of file
+covar_pop(x,y) - Returns the population covariance of a set of number pairs
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad b/sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad
index 4037062d2da93cea15ff7ad3c9b0fdaca47b4298..7323e72fccc7c92f3a3b7838839136cc619214d6 100644
--- a/sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad
+++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad
@@ -3,4 +3,4 @@ The function takes as arguments any pair of numeric types and returns a double.
 Any pair with a NULL is ignored. If the function is applied to an empty set, NULL
 will be returned. Otherwise, it computes the following:
    (SUM(x*y)-SUM(x)*SUM(y)/COUNT(x,y))/COUNT(x,y)
-where neither x nor y is null.
\ No newline at end of file
+where neither x nor y is null.
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd b/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..7951defec192aa41c72f62ac9c9f4b001cdaaba8 100644
--- a/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd
+++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd
@@ -1 +1 @@
-NULL
\ No newline at end of file
+NULL
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b b/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..7951defec192aa41c72f62ac9c9f4b001cdaaba8 100644
--- a/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b
+++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b
@@ -1 +1 @@
-NULL
\ No newline at end of file
+NULL
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae b/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae
index 171538eb0b00f4eddffa17929796de55b838f34b..ba66466c2a0d0a1080ad71d8e1a67a97f336c384 100644
--- a/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae
+++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae
@@ -1 +1 @@
-0.0
\ No newline at end of file
+0.0
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f b/sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f
index 848e15bc6147629b39e5667176417a67cabae173..104018ecd43d679e96dfc295aae25c0311b26478 100644
--- a/sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f
+++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f
@@ -3,4 +3,4 @@
 3	0.0
 4	0.0
 5	0.0
-6	0.0
\ No newline at end of file
+6	0.0
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 b/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623
index 1a49bf590b3461ff207cbc95eea052af5902aa5c..16f4e6bd601b6cbe325c72d3b70283d641dd290e 100644
--- a/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623
+++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623
@@ -1 +1 @@
-3.624999999999999
\ No newline at end of file
+3.624999999999999
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 b/sql/hive/src/test/resources/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 b/sql/hive/src/test/resources/golden/udaf_covar_samp-2-e886f45c8f085596ffd420f89cdc2909
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97
rename to sql/hive/src/test/resources/golden/udaf_covar_samp-2-e886f45c8f085596ffd420f89cdc2909
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf b/sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf
index 97a97c8b5965a39a03a697035c6b0be2297f3e06..b301d988192fd48a19eeec8cb4b1c32214080910 100644
--- a/sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf
+++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf
@@ -1 +1 @@
-covar_samp(x,y) - Returns the sample covariance of a set of number pairs
\ No newline at end of file
+covar_samp(x,y) - Returns the sample covariance of a set of number pairs
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c b/sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c
index e666adba8df68003cbaf37de7a61cb42d3c27185..de059e0e79a2738dcec964b6ee3cebe22087286a 100644
--- a/sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c
+++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c
@@ -3,4 +3,4 @@ The function takes as arguments any pair of numeric types and returns a double.
 Any pair with a NULL is ignored. If the function is applied to an empty set, NULL
 will be returned. Otherwise, it computes the following:
    (SUM(x*y)-SUM(x)*SUM(y)/COUNT(x,y))/(COUNT(x,y)-1)
-where neither x nor y is null.
\ No newline at end of file
+where neither x nor y is null.
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b b/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..7951defec192aa41c72f62ac9c9f4b001cdaaba8 100644
--- a/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b
+++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b
@@ -1 +1 @@
-NULL
\ No newline at end of file
+NULL
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 b/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..7951defec192aa41c72f62ac9c9f4b001cdaaba8 100644
--- a/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1
+++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1
@@ -1 +1 @@
-NULL
\ No newline at end of file
+NULL
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 b/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3
index 171538eb0b00f4eddffa17929796de55b838f34b..ba66466c2a0d0a1080ad71d8e1a67a97f336c384 100644
--- a/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3
+++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3
@@ -1 +1 @@
-0.0
\ No newline at end of file
+0.0
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a b/sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a
index 848e15bc6147629b39e5667176417a67cabae173..104018ecd43d679e96dfc295aae25c0311b26478 100644
--- a/sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a
+++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a
@@ -3,4 +3,4 @@
 3	0.0
 4	0.0
 5	0.0
-6	0.0
\ No newline at end of file
+6	0.0
diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 b/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683
index 30d98234a23e9c145726db4525df7479a7631c45..b8adc8f23da340b2ad3844025073a8b37fa3c8b0 100644
--- a/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683
+++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683
@@ -1 +1 @@
-4.833333333333332
\ No newline at end of file
+4.833333333333332
diff --git a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251
index 3e8bb17f24d6f44e99935d97e23e870be98a491d..df3189a887974f61ca7abbeeea13c900b7d7025c 100644
--- a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251
+++ b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251
@@ -1 +1 @@
-[{"x":135.0284552845532,"y":246.0},{"x":381.39370078740143,"y":254.0}]
\ No newline at end of file
+[{"x":135.0284552845532,"y":246.0},{"x":381.39370078740143,"y":254.0}]
diff --git a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3
index 048770d4c8402fd39f0b79b61e21511d592b4472..b1f27ead33687e0a31f4684bbf7b0b3ef45a12c1 100644
--- a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3
+++ b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3
@@ -1 +1 @@
-[{"x":96.7349397590361,"y":166.0},{"x":257.14970059880255,"y":167.0},{"x":425.6826347305388,"y":167.0}]
\ No newline at end of file
+[{"x":96.7349397590361,"y":166.0},{"x":257.14970059880255,"y":167.0},{"x":425.6826347305388,"y":167.0}]
diff --git a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1
index 8b54db8da7167bc24eceb35262d131c79f4b5e83..78a46aac82c118de5564c3bcee5d04442c80bb52 100644
--- a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1
+++ b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1
@@ -1 +1 @@
-[{"x":9.761904761904763,"y":21.0},{"x":33.84210526315789,"y":19.0},{"x":62.75000000000001,"y":20.0},{"x":90.90322580645162,"y":31.0},{"x":122.91666666666667,"y":24.0},{"x":146.33333333333334,"y":21.0},{"x":170.70967741935485,"y":31.0},{"x":194.3571428571428,"y":28.0},{"x":214.84615384615384,"y":26.0},{"x":235.08695652173907,"y":23.0},{"x":257.80000000000007,"y":15.0},{"x":281.0333333333333,"y":30.0},{"x":298.0,"y":1.0},{"x":313.0000000000001,"y":29.0},{"x":339.5925925925926,"y":27.0},{"x":372.49999999999983,"y":24.0},{"x":402.23684210526324,"y":38.0},{"x":430.6896551724138,"y":29.0},{"x":462.32352941176464,"y":34.0},{"x":487.72413793103453,"y":29.0}]
\ No newline at end of file
+[{"x":9.761904761904763,"y":21.0},{"x":33.84210526315789,"y":19.0},{"x":62.75000000000001,"y":20.0},{"x":90.90322580645162,"y":31.0},{"x":122.91666666666667,"y":24.0},{"x":146.33333333333334,"y":21.0},{"x":170.70967741935485,"y":31.0},{"x":194.3571428571428,"y":28.0},{"x":214.84615384615384,"y":26.0},{"x":235.08695652173907,"y":23.0},{"x":257.80000000000007,"y":15.0},{"x":281.0333333333333,"y":30.0},{"x":298.0,"y":1.0},{"x":313.0000000000001,"y":29.0},{"x":339.5925925925926,"y":27.0},{"x":372.49999999999983,"y":24.0},{"x":402.23684210526324,"y":38.0},{"x":430.6896551724138,"y":29.0},{"x":462.32352941176464,"y":34.0},{"x":487.72413793103453,"y":29.0}]
diff --git a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6
index aed3f1e704750a1fd187e0d99a30188524cf9bc8..4f7995f874388ed0dff86cb3f7e090b8df06ffbf 100644
--- a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6
+++ b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6
@@ -1 +1 @@
-[{"x":0.0,"y":3.0},{"x":2.0,"y":1.0},{"x":4.75,"y":4.0},{"x":8.0,"y":1.0},{"x":9.5,"y":2.0},{"x":11.666666666666666,"y":3.0},{"x":15.0,"y":2.0},{"x":17.666666666666664,"y":3.0},{"x":19.5,"y":2.0},{"x":24.0,"y":2.0},{"x":26.333333333333336,"y":3.0},{"x":28.0,"y":1.0},{"x":30.0,"y":1.0},{"x":33.0,"y":1.0},{"x":34.75,"y":4.0},{"x":37.0,"y":2.0},{"x":41.666666666666664,"y":3.0},{"x":43.5,"y":2.0},{"x":47.0,"y":1.0},{"x":51.0,"y":2.0},{"x":53.5,"y":2.0},{"x":57.666666666666664,"y":3.0},{"x":64.5,"y":2.0},{"x":66.66666666666666,"y":3.0},{"x":69.75,"y":4.0},{"x":72.0,"y":2.0},{"x":74.0,"y":1.0},{"x":76.33333333333333,"y":3.0},{"x":78.0,"y":1.0},{"x":80.0,"y":1.0},{"x":82.0,"y":1.0},{"x":83.5,"y":4.0},{"x":85.5,"y":2.0},{"x":87.0,"y":1.0},{"x":90.0,"y":3.0},{"x":92.0,"y":1.0},{"x":95.33333333333333,"y":3.0},{"x":97.5,"y":4.0},{"x":100.0,"y":2.0},{"x":103.5,"y":4.0},{"x":105.0,"y":1.0},{"x":111.0,"y":1.0},{"x":113.33333333333333,"y":3.0},{"x":116.0,"y":1.0},{"x":118.0,"y":2.0},{"x":119.4,"y":5.0},{"x":125.33333333333333,"y":3.0},{"x":128.4,"y":5.0},{"x":131.0,"y":1.0},{"x":133.66666666666666,"y":3.0},{"x":136.66666666666666,"y":3.0},{"x":138.0,"y":4.0},{"x":143.0,"y":1.0},{"x":145.66666666666666,"y":3.0},{"x":149.33333333333331,"y":3.0},{"x":152.33333333333334,"y":3.0},{"x":155.5,"y":2.0},{"x":157.5,"y":2.0},{"x":160.0,"y":1.0},{"x":162.5,"y":2.0},{"x":164.5,"y":4.0},{"x":166.75,"y":4.0},{"x":168.8,"y":5.0},{"x":170.0,"y":1.0},{"x":172.0,"y":2.0},{"x":174.5,"y":4.0},{"x":176.33333333333331,"y":3.0},{"x":178.0,"y":1.0},{"x":179.33333333333331,"y":3.0},{"x":181.0,"y":1.0},{"x":183.0,"y":1.0},{"x":186.75,"y":4.0},{"x":189.0,"y":1.0},{"x":190.66666666666666,"y":3.0},{"x":192.75,"y":4.0},{"x":194.0,"y":1.0},{"x":195.33333333333331,"y":3.0},{"x":197.0,"y":2.0},{"x":199.4,"y":5.0},{"x":201.0,"y":1.0},{"x":202.66666666666669,"y":3.0},{"x":205.0,"y":2.0},{"x":207.0,"y":2.0},{"x":208.40000000000003,"y":5.0},{"x":213.33333333333331,"y":3.0},{"x":216.0,"y":2.0},{"x":217.33333333333331,"y":3.0},{"x":219.0,"y":2.0},{"x":221.33333333333331,"y":3.0},{"x":223.5,"y":4.0},{"x":226.0,"y":1.0},{"x":228.66666666666663,"y":3.0},{"x":230.0,"y":5.0},{"x":233.0,"y":2.0},{"x":235.0,"y":1.0},{"x":237.5,"y":4.0},{"x":239.0,"y":2.0},{"x":241.66666666666669,"y":3.0},{"x":244.0,"y":1.0},{"x":247.5,"y":2.0},{"x":249.0,"y":1.0},{"x":252.0,"y":1.0},{"x":255.5,"y":4.0},{"x":257.5,"y":2.0},{"x":260.0,"y":1.0},{"x":262.5,"y":2.0},{"x":265.3333333333333,"y":3.0},{"x":272.6,"y":5.0},{"x":274.5,"y":2.0},{"x":277.3333333333333,"y":6.0},{"x":280.0,"y":2.0},{"x":281.5,"y":4.0},{"x":283.5,"y":2.0},{"x":285.0,"y":1.0},{"x":286.5,"y":2.0},{"x":288.3333333333333,"y":3.0},{"x":291.5,"y":2.0},{"x":296.0,"y":1.0},{"x":298.0,"y":3.0},{"x":302.0,"y":1.0},{"x":305.5,"y":2.0},{"x":307.3333333333333,"y":3.0},{"x":309.0,"y":2.0},{"x":310.75,"y":4.0},{"x":315.75,"y":4.0},{"x":317.6,"y":5.0},{"x":321.5,"y":4.0},{"x":323.0,"y":1.0},{"x":325.0,"y":2.0},{"x":327.0,"y":3.0},{"x":331.3333333333333,"y":3.0},{"x":333.0,"y":2.0},{"x":335.5,"y":2.0},{"x":338.5,"y":2.0},{"x":341.66666666666663,"y":3.0},{"x":344.3333333333333,"y":3.0},{"x":348.0,"y":5.0},{"x":351.0,"y":1.0},{"x":353.0,"y":2.0},{"x":356.0,"y":1.0},{"x":360.0,"y":1.0},{"x":362.0,"y":1.0},{"x":364.5,"y":2.0},{"x":366.66666666666663,"y":3.0},{"x":368.75,"y":4.0},{"x":373.5,"y":2.0},{"x":375.0,"y":1.0},{"x":377.5,"y":2.0},{"x":379.0,"y":1.0},{"x":382.0,"y":2.0},{"x":384.0,"y":3.0},{"x":386.0,"y":1.0},{"x":389.0,"y":1.0},{"x":392.0,"y":1.0},{"x":393.5,"y":2.0},{"x":395.6,"y":5.0},{"x":397.0,"y":2.0},{"x":399.0,"y":2.0},{"x":400.0,"y":1.0},{"x":401.16666666666663,"y":6.0},{"x":403.40000000000003,"y":5.0},{"x":406.20000000000005,"y":5.0},{"x":409.0,"y":3.0},{"x":411.0,"y":1.0},{"x":413.5,"y":4.0},{"x":417.0,"y":3.0},{"x":418.5,"y":2.0},{"x":421.0,"y":1.0},{"x":424.0,"y":2.0},{"x":427.0,"y":1.0},{"x":429.6,"y":5.0},{"x":431.25,"y":4.0},{"x":435.5,"y":2.0},{"x":437.75,"y":4.0},{"x":439.0,"y":2.0},{"x":443.5,"y":2.0},{"x":446.0,"y":1.0},{"x":448.5,"y":2.0},{"x":452.5,"y":2.0},{"x":454.24999999999994,"y":4.0},{"x":457.66666666666663,"y":3.0},{"x":459.33333333333337,"y":3.0},{"x":462.5,"y":4.0},{"x":466.0,"y":3.0},{"x":467.80000000000007,"y":5.0},{"x":469.16666666666663,"y":6.0},{"x":472.0,"y":1.0},{"x":475.0,"y":1.0},{"x":477.0,"y":1.0},{"x":478.33333333333326,"y":3.0},{"x":480.25,"y":4.0},{"x":482.5,"y":2.0},{"x":484.5,"y":2.0},{"x":487.0,"y":1.0},{"x":489.2,"y":5.0},{"x":491.66666666666663,"y":3.0},{"x":493.0,"y":1.0},{"x":494.5,"y":2.0},{"x":496.0,"y":1.0},{"x":497.75,"y":4.0}]
\ No newline at end of file
+[{"x":0.0,"y":3.0},{"x":2.0,"y":1.0},{"x":4.75,"y":4.0},{"x":8.0,"y":1.0},{"x":9.5,"y":2.0},{"x":11.666666666666666,"y":3.0},{"x":15.0,"y":2.0},{"x":17.666666666666664,"y":3.0},{"x":19.5,"y":2.0},{"x":24.0,"y":2.0},{"x":26.333333333333336,"y":3.0},{"x":28.0,"y":1.0},{"x":30.0,"y":1.0},{"x":33.0,"y":1.0},{"x":34.75,"y":4.0},{"x":37.0,"y":2.0},{"x":41.666666666666664,"y":3.0},{"x":43.5,"y":2.0},{"x":47.0,"y":1.0},{"x":51.0,"y":2.0},{"x":53.5,"y":2.0},{"x":57.666666666666664,"y":3.0},{"x":64.5,"y":2.0},{"x":66.66666666666666,"y":3.0},{"x":69.75,"y":4.0},{"x":72.0,"y":2.0},{"x":74.0,"y":1.0},{"x":76.33333333333333,"y":3.0},{"x":78.0,"y":1.0},{"x":80.0,"y":1.0},{"x":82.0,"y":1.0},{"x":83.5,"y":4.0},{"x":85.5,"y":2.0},{"x":87.0,"y":1.0},{"x":90.0,"y":3.0},{"x":92.0,"y":1.0},{"x":95.33333333333333,"y":3.0},{"x":97.5,"y":4.0},{"x":100.0,"y":2.0},{"x":103.5,"y":4.0},{"x":105.0,"y":1.0},{"x":111.0,"y":1.0},{"x":113.33333333333333,"y":3.0},{"x":116.0,"y":1.0},{"x":118.0,"y":2.0},{"x":119.4,"y":5.0},{"x":125.33333333333333,"y":3.0},{"x":128.4,"y":5.0},{"x":131.0,"y":1.0},{"x":133.66666666666666,"y":3.0},{"x":136.66666666666666,"y":3.0},{"x":138.0,"y":4.0},{"x":143.0,"y":1.0},{"x":145.66666666666666,"y":3.0},{"x":149.33333333333331,"y":3.0},{"x":152.33333333333334,"y":3.0},{"x":155.5,"y":2.0},{"x":157.5,"y":2.0},{"x":160.0,"y":1.0},{"x":162.5,"y":2.0},{"x":164.5,"y":4.0},{"x":166.75,"y":4.0},{"x":168.8,"y":5.0},{"x":170.0,"y":1.0},{"x":172.0,"y":2.0},{"x":174.5,"y":4.0},{"x":176.33333333333331,"y":3.0},{"x":178.0,"y":1.0},{"x":179.33333333333331,"y":3.0},{"x":181.0,"y":1.0},{"x":183.0,"y":1.0},{"x":186.75,"y":4.0},{"x":189.0,"y":1.0},{"x":190.66666666666666,"y":3.0},{"x":192.75,"y":4.0},{"x":194.0,"y":1.0},{"x":195.33333333333331,"y":3.0},{"x":197.0,"y":2.0},{"x":199.4,"y":5.0},{"x":201.0,"y":1.0},{"x":202.66666666666669,"y":3.0},{"x":205.0,"y":2.0},{"x":207.0,"y":2.0},{"x":208.40000000000003,"y":5.0},{"x":213.33333333333331,"y":3.0},{"x":216.0,"y":2.0},{"x":217.33333333333331,"y":3.0},{"x":219.0,"y":2.0},{"x":221.33333333333331,"y":3.0},{"x":223.5,"y":4.0},{"x":226.0,"y":1.0},{"x":228.66666666666663,"y":3.0},{"x":230.0,"y":5.0},{"x":233.0,"y":2.0},{"x":235.0,"y":1.0},{"x":237.5,"y":4.0},{"x":239.0,"y":2.0},{"x":241.66666666666669,"y":3.0},{"x":244.0,"y":1.0},{"x":247.5,"y":2.0},{"x":249.0,"y":1.0},{"x":252.0,"y":1.0},{"x":255.5,"y":4.0},{"x":257.5,"y":2.0},{"x":260.0,"y":1.0},{"x":262.5,"y":2.0},{"x":265.3333333333333,"y":3.0},{"x":272.6,"y":5.0},{"x":274.5,"y":2.0},{"x":277.3333333333333,"y":6.0},{"x":280.0,"y":2.0},{"x":281.5,"y":4.0},{"x":283.5,"y":2.0},{"x":285.0,"y":1.0},{"x":286.5,"y":2.0},{"x":288.3333333333333,"y":3.0},{"x":291.5,"y":2.0},{"x":296.0,"y":1.0},{"x":298.0,"y":3.0},{"x":302.0,"y":1.0},{"x":305.5,"y":2.0},{"x":307.3333333333333,"y":3.0},{"x":309.0,"y":2.0},{"x":310.75,"y":4.0},{"x":315.75,"y":4.0},{"x":317.6,"y":5.0},{"x":321.5,"y":4.0},{"x":323.0,"y":1.0},{"x":325.0,"y":2.0},{"x":327.0,"y":3.0},{"x":331.3333333333333,"y":3.0},{"x":333.0,"y":2.0},{"x":335.5,"y":2.0},{"x":338.5,"y":2.0},{"x":341.66666666666663,"y":3.0},{"x":344.3333333333333,"y":3.0},{"x":348.0,"y":5.0},{"x":351.0,"y":1.0},{"x":353.0,"y":2.0},{"x":356.0,"y":1.0},{"x":360.0,"y":1.0},{"x":362.0,"y":1.0},{"x":364.5,"y":2.0},{"x":366.66666666666663,"y":3.0},{"x":368.75,"y":4.0},{"x":373.5,"y":2.0},{"x":375.0,"y":1.0},{"x":377.5,"y":2.0},{"x":379.0,"y":1.0},{"x":382.0,"y":2.0},{"x":384.0,"y":3.0},{"x":386.0,"y":1.0},{"x":389.0,"y":1.0},{"x":392.0,"y":1.0},{"x":393.5,"y":2.0},{"x":395.6,"y":5.0},{"x":397.0,"y":2.0},{"x":399.0,"y":2.0},{"x":400.0,"y":1.0},{"x":401.16666666666663,"y":6.0},{"x":403.40000000000003,"y":5.0},{"x":406.20000000000005,"y":5.0},{"x":409.0,"y":3.0},{"x":411.0,"y":1.0},{"x":413.5,"y":4.0},{"x":417.0,"y":3.0},{"x":418.5,"y":2.0},{"x":421.0,"y":1.0},{"x":424.0,"y":2.0},{"x":427.0,"y":1.0},{"x":429.6,"y":5.0},{"x":431.25,"y":4.0},{"x":435.5,"y":2.0},{"x":437.75,"y":4.0},{"x":439.0,"y":2.0},{"x":443.5,"y":2.0},{"x":446.0,"y":1.0},{"x":448.5,"y":2.0},{"x":452.5,"y":2.0},{"x":454.24999999999994,"y":4.0},{"x":457.66666666666663,"y":3.0},{"x":459.33333333333337,"y":3.0},{"x":462.5,"y":4.0},{"x":466.0,"y":3.0},{"x":467.80000000000007,"y":5.0},{"x":469.16666666666663,"y":6.0},{"x":472.0,"y":1.0},{"x":475.0,"y":1.0},{"x":477.0,"y":1.0},{"x":478.33333333333326,"y":3.0},{"x":480.25,"y":4.0},{"x":482.5,"y":2.0},{"x":484.5,"y":2.0},{"x":487.0,"y":1.0},{"x":489.2,"y":5.0},{"x":491.66666666666663,"y":3.0},{"x":493.0,"y":1.0},{"x":494.5,"y":2.0},{"x":496.0,"y":1.0},{"x":497.75,"y":4.0}]
diff --git a/sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 b/sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 b/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16
deleted file mode 100644
index 2953abcf1e64488cc9d9559359960ffeaf5a0bd4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16
+++ /dev/null
@@ -1 +0,0 @@
-0.0	NULL	NULL	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff
deleted file mode 100644
index 17c47d308f029de1aa1886d78b6a7f28ee711161..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff
+++ /dev/null
@@ -1 +0,0 @@
-254.08333333333334
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3
deleted file mode 100644
index 17c47d308f029de1aa1886d78b6a7f28ee711161..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3
+++ /dev/null
@@ -1 +0,0 @@
-254.08333333333334
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1
deleted file mode 100644
index 014c31564909642db05c520120867c762007694f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1
+++ /dev/null
@@ -1 +0,0 @@
-[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c
deleted file mode 100644
index 014c31564909642db05c520120867c762007694f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c
+++ /dev/null
@@ -1 +0,0 @@
-[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff
deleted file mode 100644
index 17c47d308f029de1aa1886d78b6a7f28ee711161..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff
+++ /dev/null
@@ -1 +0,0 @@
-254.08333333333334
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3
deleted file mode 100644
index 17c47d308f029de1aa1886d78b6a7f28ee711161..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3
+++ /dev/null
@@ -1 +0,0 @@
-254.08333333333334
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1
deleted file mode 100644
index 014c31564909642db05c520120867c762007694f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1
+++ /dev/null
@@ -1 +0,0 @@
-[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c
deleted file mode 100644
index 014c31564909642db05c520120867c762007694f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c
+++ /dev/null
@@ -1 +0,0 @@
-[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff
deleted file mode 100644
index 17c47d308f029de1aa1886d78b6a7f28ee711161..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff
+++ /dev/null
@@ -1 +0,0 @@
-254.08333333333334
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3
deleted file mode 100644
index 17c47d308f029de1aa1886d78b6a7f28ee711161..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3
+++ /dev/null
@@ -1 +0,0 @@
-254.08333333333334
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1
deleted file mode 100644
index 014c31564909642db05c520120867c762007694f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1
+++ /dev/null
@@ -1 +0,0 @@
-[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c
deleted file mode 100644
index 014c31564909642db05c520120867c762007694f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c
+++ /dev/null
@@ -1 +0,0 @@
-[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff
deleted file mode 100644
index 17c47d308f029de1aa1886d78b6a7f28ee711161..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff
+++ /dev/null
@@ -1 +0,0 @@
-254.08333333333334
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3
deleted file mode 100644
index 17c47d308f029de1aa1886d78b6a7f28ee711161..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3
+++ /dev/null
@@ -1 +0,0 @@
-254.08333333333334
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51
deleted file mode 100644
index 60cbe79310729ad29082730acfab232b03e99e1a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51
+++ /dev/null
@@ -1 +0,0 @@
-255.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1
deleted file mode 100644
index 014c31564909642db05c520120867c762007694f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1
+++ /dev/null
@@ -1 +0,0 @@
-[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c
deleted file mode 100644
index 014c31564909642db05c520120867c762007694f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c
+++ /dev/null
@@ -1 +0,0 @@
-[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28
deleted file mode 100644
index a8986e32ff75abe25e1e204c5f42e3f74f2957ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28
+++ /dev/null
@@ -1 +0,0 @@
-[26.0,255.5,479.0,491.0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 b/sql/hive/src/test/resources/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf1-1-8281592c818ada269024ac669bec78da b/sql/hive/src/test/resources/golden/udf1-1-8281592c818ada269024ac669bec78da
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f b/sql/hive/src/test/resources/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index 493daf5d79c54821b4d05cf27442d639b2f8c19d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1 +0,0 @@
-true	false	true	true	true	false	false	false	true	true	false	true	true	acc	abc	abb	hive	hadoop	AaAbAcA	false
diff --git a/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 b/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591
index 398f517b25b5fb878b84330323180864b35bc137..2d3e77d99bca3b59d116eab5163ae2a2249906fc 100644
--- a/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591
+++ b/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591
@@ -1 +1 @@
-|	abc	|	  abc	|	abc  	|
\ No newline at end of file
+|	abc	|	  abc	|	abc  	|
diff --git a/sql/hive/src/test/resources/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c b/sql/hive/src/test/resources/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 b/sql/hive/src/test/resources/golden/udf3-1-1d04874d496d05cfe0b9d86de1111
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e b/sql/hive/src/test/resources/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index b2890dc4cdb3ac7d9f532c04abee61390e85921f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1 +0,0 @@
-0	NULL	NULL	NULL	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73 b/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73
index 56a6051ca2b02b04ef92d5150c9ef600403cb1de..d00491fd7e5bb6fa28c517a0bb32b8b506539d4d 100644
--- a/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73
+++ b/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73
@@ -1 +1 @@
-1
\ No newline at end of file
+1
diff --git a/sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e b/sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e
index 132dbea91f8a0a0287d99d93d7f5d3f83a5665e4..5657917e40f7d7b82e79b5df05a4c66649411782 100644
--- a/sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e
+++ b/sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e
@@ -1 +1 @@
-1	2	2	a	0.1	2	126	128	128	1.0	128
\ No newline at end of file
+1	2	2	a	0.1	2	126	128	128	1.0	128
diff --git a/sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 b/sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045
index 51958ff45ab8d7d3a9bc14539123dc33b1c2a29f..2fcaff3dad9fe0c52f1ace46adab75947e2a93da 100644
--- a/sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045
+++ b/sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045
@@ -1 +1 @@
-1.098612288668	NULL	NULL	1.098612288668	NULL	NULL	1.584962500721	NULL	NULL	0.47712125472	NULL	NULL	1.584962500721	NULL	NULL	NULL	-1.0	7.389056098931	8.0	8.0	0.125	8.0	2.0	NaN	1.0	1	8	8
+1.098612288668	NULL	NULL	1.098612288668	NULL	NULL	1.584962500721	NULL	NULL	0.47712125472	NULL	NULL	1.584962500721	NULL	NULL	NULL	-1.0	7.389056098931	8.0	8.0	0.125	8.0	2.0	NaN	1.0	1.0	8.0	8.0
diff --git a/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b b/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b
index 8a89b039b71510d0a36740969f6d62c36beb3012..60b09adaf8029c00d54a02654ee457da032293e9 100644
--- a/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b
+++ b/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b
@@ -1 +1 @@
-1.0	1.0	1
\ No newline at end of file
+1.0	1.0	1
diff --git a/sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c b/sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c
index 83e2e121222a24e649c6ff6899bf760aba2cc265..cb93f99495494e48ea8b6ae598df132ff72b8cc1 100644
--- a/sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c
+++ b/sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c
@@ -1 +1 @@
--1	2	32	-1	2009-01-01	2009-12-31	2008-03-01	2009-03-02	2008-02-28	2009-02-27	2008-12-31	2008-01-02	2008-02-28	2009-02-27	2006-02-28	2005-02-28
\ No newline at end of file
+-1	2	32	-1	2009-01-01	2009-12-31	2008-03-01	2009-03-02	2008-02-28	2009-02-27	2008-12-31	2008-01-02	2008-02-28	2009-02-27	2006-02-28	2005-02-28
diff --git a/sql/hive/src/test/resources/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f b/sql/hive/src/test/resources/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udf_E-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/udf_E-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/udf_E-1-cad0779d18f326c8e453bf2b5fe43596
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b
rename to sql/hive/src/test/resources/golden/udf_E-1-cad0779d18f326c8e453bf2b5fe43596
diff --git a/sql/hive/src/test/resources/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c b/sql/hive/src/test/resources/golden/udf_E-2-d0fd9aa04fdeb948bdcf8559f7095c02
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c
rename to sql/hive/src/test/resources/golden/udf_E-2-d0fd9aa04fdeb948bdcf8559f7095c02
diff --git a/sql/hive/src/test/resources/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a b/sql/hive/src/test/resources/golden/udf_E-3-72bb4231ea2a877b4d93a53cd7b6b82a
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a
rename to sql/hive/src/test/resources/golden/udf_E-3-72bb4231ea2a877b4d93a53cd7b6b82a
diff --git a/sql/hive/src/test/resources/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f b/sql/hive/src/test/resources/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 b/sql/hive/src/test/resources/golden/udf_E-4-e8924af3bf99d2e01546a965303ffd09
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09
rename to sql/hive/src/test/resources/golden/udf_E-4-e8924af3bf99d2e01546a965303ffd09
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 b/sql/hive/src/test/resources/golden/udf_E-5-9d54c12bf727e05e9f9d67c61402a1d4
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8
rename to sql/hive/src/test/resources/golden/udf_E-5-9d54c12bf727e05e9f9d67c61402a1d4
diff --git a/sql/hive/src/test/resources/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c b/sql/hive/src/test/resources/golden/udf_E-6-d0fd9aa04fdeb948bdcf8559f7095c02
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c
rename to sql/hive/src/test/resources/golden/udf_E-6-d0fd9aa04fdeb948bdcf8559f7095c02
diff --git a/sql/hive/src/test/resources/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a b/sql/hive/src/test/resources/golden/udf_E-7-72bb4231ea2a877b4d93a53cd7b6b82a
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a
rename to sql/hive/src/test/resources/golden/udf_E-7-72bb4231ea2a877b4d93a53cd7b6b82a
diff --git a/sql/hive/src/test/resources/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 b/sql/hive/src/test/resources/golden/udf_E-8-e8924af3bf99d2e01546a965303ffd09
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09
rename to sql/hive/src/test/resources/golden/udf_E-8-e8924af3bf99d2e01546a965303ffd09
diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_PI-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/udf_PI-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f b/sql/hive/src/test/resources/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 b/sql/hive/src/test/resources/golden/udf_PI-1-13fd9345fd15b654d18b2707e5274b2b
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9
rename to sql/hive/src/test/resources/golden/udf_PI-1-13fd9345fd15b654d18b2707e5274b2b
diff --git a/sql/hive/src/test/resources/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 b/sql/hive/src/test/resources/golden/udf_PI-2-97a12f6967726e425469ecfa70177ff0
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7
rename to sql/hive/src/test/resources/golden/udf_PI-2-97a12f6967726e425469ecfa70177ff0
diff --git a/sql/hive/src/test/resources/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 b/sql/hive/src/test/resources/golden/udf_PI-3-9c1476a2eab7455594e97b338ee3c188
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188
rename to sql/hive/src/test/resources/golden/udf_PI-3-9c1476a2eab7455594e97b338ee3c188
diff --git a/sql/hive/src/test/resources/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 b/sql/hive/src/test/resources/golden/udf_PI-4-890f3c276eff2c459d8dc79d5a71c866
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866
rename to sql/hive/src/test/resources/golden/udf_PI-4-890f3c276eff2c459d8dc79d5a71c866
diff --git a/sql/hive/src/test/resources/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f b/sql/hive/src/test/resources/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 b/sql/hive/src/test/resources/golden/udf_PI-5-cd1c31c39277a02bab8e44f8c29a6c2d
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37
rename to sql/hive/src/test/resources/golden/udf_PI-5-cd1c31c39277a02bab8e44f8c29a6c2d
diff --git a/sql/hive/src/test/resources/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 b/sql/hive/src/test/resources/golden/udf_PI-6-97a12f6967726e425469ecfa70177ff0
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7
rename to sql/hive/src/test/resources/golden/udf_PI-6-97a12f6967726e425469ecfa70177ff0
diff --git a/sql/hive/src/test/resources/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 b/sql/hive/src/test/resources/golden/udf_PI-7-9c1476a2eab7455594e97b338ee3c188
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188
rename to sql/hive/src/test/resources/golden/udf_PI-7-9c1476a2eab7455594e97b338ee3c188
diff --git a/sql/hive/src/test/resources/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 b/sql/hive/src/test/resources/golden/udf_PI-8-890f3c276eff2c459d8dc79d5a71c866
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866
rename to sql/hive/src/test/resources/golden/udf_PI-8-890f3c276eff2c459d8dc79d5a71c866
diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udf_abs-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/udf_abs-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 b/sql/hive/src/test/resources/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234
deleted file mode 100644
index b613b3b9fe96780dce5e5d2022d5850979a7c379..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234
+++ /dev/null
@@ -1 +0,0 @@
-abs(x) - returns the absolute value of x
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_abs-1-6fe2e69c338fc823d3f61c9236eb2234 b/sql/hive/src/test/resources/golden/udf_abs-1-6fe2e69c338fc823d3f61c9236eb2234
new file mode 100644
index 0000000000000000000000000000000000000000..f1e01bf0d2fc8bba1db1bc03130ad09bce0c2d18
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_abs-1-6fe2e69c338fc823d3f61c9236eb2234
@@ -0,0 +1 @@
+abs(x) - returns the absolute value of x
diff --git a/sql/hive/src/test/resources/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 b/sql/hive/src/test/resources/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072
deleted file mode 100644
index c2fb6dc49322d751fa3f58a1dd6651a75369f698..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072
+++ /dev/null
@@ -1,6 +0,0 @@
-abs(x) - returns the absolute value of x
-Example:
-  > SELECT abs(0) FROM src LIMIT 1;
-  0
-  > SELECT abs(-5) FROM src LIMIT 1;
-  5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 b/sql/hive/src/test/resources/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_abs-2-eeb77ae8a0dcebbc0991923ca0932072 b/sql/hive/src/test/resources/golden/udf_abs-2-eeb77ae8a0dcebbc0991923ca0932072
new file mode 100644
index 0000000000000000000000000000000000000000..7dea2445fa61641acf4650fa1e709c0c28f6a49a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_abs-2-eeb77ae8a0dcebbc0991923ca0932072
@@ -0,0 +1,6 @@
+abs(x) - returns the absolute value of x
+Example:
+  > SELECT abs(0) FROM src LIMIT 1;
+  0
+  > SELECT abs(-5) FROM src LIMIT 1;
+  5
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 b/sql/hive/src/test/resources/golden/udf_abs-3-50cb3c23902cd29e0dbff188c71062e5
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715
rename to sql/hive/src/test/resources/golden/udf_abs-3-50cb3c23902cd29e0dbff188c71062e5
diff --git a/sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 b/sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530
deleted file mode 100644
index f5a78bc6e59b89b29ce06d792052f0389dcc77aa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530
+++ /dev/null
@@ -1 +0,0 @@
-0	1	123	9223372036854775807	9223372036854775807
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c b/sql/hive/src/test/resources/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_abs-4-4ae7f62f8d996f0066037cecbf2e01c4 b/sql/hive/src/test/resources/golden/udf_abs-4-4ae7f62f8d996f0066037cecbf2e01c4
new file mode 100644
index 0000000000000000000000000000000000000000..c7b1f75a06cb3f4717fdfc4a56d5f23c18085126
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_abs-4-4ae7f62f8d996f0066037cecbf2e01c4
@@ -0,0 +1 @@
+0	1	123	9223372036854775807	9223372036854775807
diff --git a/sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 b/sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4
deleted file mode 100644
index 9e57fa65e8014ec030b01b8f676911ec68df8540..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4
+++ /dev/null
@@ -1 +0,0 @@
-0.0	3.14159265	3.14159265
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 b/sql/hive/src/test/resources/golden/udf_abs-5-5cd4c198e0de884ad436864b95fece6c
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7
rename to sql/hive/src/test/resources/golden/udf_abs-5-5cd4c198e0de884ad436864b95fece6c
diff --git a/sql/hive/src/test/resources/golden/udf_abs-6-7aa32a019499c6464aded2e357c6843b b/sql/hive/src/test/resources/golden/udf_abs-6-7aa32a019499c6464aded2e357c6843b
new file mode 100644
index 0000000000000000000000000000000000000000..a07dd4e9d970de3d4657de0e1d8eb2bd771f30f2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_abs-6-7aa32a019499c6464aded2e357c6843b
@@ -0,0 +1 @@
+0.0	3.14159265	3.14159265
diff --git a/sql/hive/src/test/resources/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 b/sql/hive/src/test/resources/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06
deleted file mode 100644
index 204db415144a71d39c6fef8e894cee08bdb83a49..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06
+++ /dev/null
@@ -1 +0,0 @@
-acos(x) - returns the arc cosine of x if -1<=x<=1 or NULL otherwise
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/udf_acos-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e
rename to sql/hive/src/test/resources/golden/udf_acos-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_acos-1-4f49cb5a5c87efea534d63ed76435f06 b/sql/hive/src/test/resources/golden/udf_acos-1-4f49cb5a5c87efea534d63ed76435f06
new file mode 100644
index 0000000000000000000000000000000000000000..b32b4244d6aeb12dc932388b5bce8e626e713aff
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_acos-1-4f49cb5a5c87efea534d63ed76435f06
@@ -0,0 +1 @@
+acos(x) - returns the arc cosine of x if -1<=x<=1 or NULL otherwise
diff --git a/sql/hive/src/test/resources/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f b/sql/hive/src/test/resources/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f
deleted file mode 100644
index d49af1cbf4800b205e5188bd769ab038af20c970..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f
+++ /dev/null
@@ -1,6 +0,0 @@
-acos(x) - returns the arc cosine of x if -1<=x<=1 or NULL otherwise
-Example:
-  > SELECT acos(1) FROM src LIMIT 1;
-  0
-  > SELECT acos(2) FROM src LIMIT 1;
-  NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b b/sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_acos-2-d330511cf4f626cd844b18f57f99a85f b/sql/hive/src/test/resources/golden/udf_acos-2-d330511cf4f626cd844b18f57f99a85f
new file mode 100644
index 0000000000000000000000000000000000000000..93925317cd7987fc3f6dc3e2369839bc31796143
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_acos-2-d330511cf4f626cd844b18f57f99a85f
@@ -0,0 +1,6 @@
+acos(x) - returns the arc cosine of x if -1<=x<=1 or NULL otherwise
+Example:
+  > SELECT acos(1) FROM src LIMIT 1;
+  0
+  > SELECT acos(2) FROM src LIMIT 1;
+  NULL
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 b/sql/hive/src/test/resources/golden/udf_acos-3-661a0a85283df2a5c1567d60850e362b
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0
rename to sql/hive/src/test/resources/golden/udf_acos-3-661a0a85283df2a5c1567d60850e362b
diff --git a/sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 b/sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884
deleted file mode 100644
index 5548bdb7cf26a9a60c6aabf213bcc8b13b6e7dd6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884
+++ /dev/null
@@ -1 +0,0 @@
-1.5707963267948966
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_acos-4-23d588eece08fbea7431044524f1cecf b/sql/hive/src/test/resources/golden/udf_acos-4-23d588eece08fbea7431044524f1cecf
new file mode 100644
index 0000000000000000000000000000000000000000..53a6e14702ed1fe854dc26a14991e0e5d6026c82
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_acos-4-23d588eece08fbea7431044524f1cecf
@@ -0,0 +1 @@
+1.5707963267948966
diff --git a/sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a b/sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a
deleted file mode 100644
index 1f2d6faad9a2cce178e02ee7d1e784b5705538ea..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a
+++ /dev/null
@@ -1 +0,0 @@
-2.0943951023931957	0.7208187608700897
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_acos-5-578612589fdb1ae21ee488924848fb4e b/sql/hive/src/test/resources/golden/udf_acos-5-578612589fdb1ae21ee488924848fb4e
new file mode 100644
index 0000000000000000000000000000000000000000..edee342d1d902f94a2b5689140d8c774cb525bfd
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_acos-5-578612589fdb1ae21ee488924848fb4e
@@ -0,0 +1 @@
+2.0943951023931957	0.7208187608700897
diff --git a/sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e b/sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e
deleted file mode 100644
index b955cad2a39a09a1c4143ccffb078ac94bb80b4c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e
+++ /dev/null
@@ -1 +0,0 @@
-NaN
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_acos-6-4d2bd33cee047e9a8bb740760c7cc3b4 b/sql/hive/src/test/resources/golden/udf_acos-6-4d2bd33cee047e9a8bb740760c7cc3b4
new file mode 100644
index 0000000000000000000000000000000000000000..736991a13874574828c7a0fdd2ead5aac88cac65
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_acos-6-4d2bd33cee047e9a8bb740760c7cc3b4
@@ -0,0 +1 @@
+NaN
diff --git a/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 b/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045
index e8f11c444a808d7aae34306e037017d597d783d5..9d794539b4ae0a5393e482ec39ca108997643290 100644
--- a/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045
+++ b/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045
@@ -1 +1 @@
-a + b - Returns a+b
\ No newline at end of file
+a + b - Returns a+b
diff --git a/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 b/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5
index e8f11c444a808d7aae34306e037017d597d783d5..9d794539b4ae0a5393e482ec39ca108997643290 100644
--- a/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5
+++ b/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5
@@ -1 +1 @@
-a + b - Returns a+b
\ No newline at end of file
+a + b - Returns a+b
diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_array-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/udf_array-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee b/sql/hive/src/test/resources/golden/udf_array-1-2e9c2a7d9325bd9a53cf9f181b6333ee
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee
rename to sql/hive/src/test/resources/golden/udf_array-1-2e9c2a7d9325bd9a53cf9f181b6333ee
diff --git a/sql/hive/src/test/resources/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a b/sql/hive/src/test/resources/golden/udf_array-2-570741914bb78300b0233e5f38d7f08a
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a
rename to sql/hive/src/test/resources/golden/udf_array-2-570741914bb78300b0233e5f38d7f08a
diff --git a/sql/hive/src/test/resources/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 b/sql/hive/src/test/resources/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc b/sql/hive/src/test/resources/golden/udf_array-3-47818d42e5e7667d8754c3f9a4b8053a
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc
rename to sql/hive/src/test/resources/golden/udf_array-3-47818d42e5e7667d8754c3f9a4b8053a
diff --git a/sql/hive/src/test/resources/golden/udf_array-3-a5d12c41277fb158e09281169c905122 b/sql/hive/src/test/resources/golden/udf_array-4-51410e4d4d679fe5a8dd7a860f4efc47
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_array-3-a5d12c41277fb158e09281169c905122
rename to sql/hive/src/test/resources/golden/udf_array-4-51410e4d4d679fe5a8dd7a860f4efc47
diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udf_array_contains-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/udf_array_contains-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a b/sql/hive/src/test/resources/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a
deleted file mode 100644
index 783e7d086a5c69264e46b01747fc24d6f8da4961..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a
+++ /dev/null
@@ -1 +0,0 @@
-array_contains(array, value) - Returns TRUE if the array contains value.
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-1-d9a90108b052b111e8de4433e008b25a b/sql/hive/src/test/resources/golden/udf_array_contains-1-d9a90108b052b111e8de4433e008b25a
new file mode 100644
index 0000000000000000000000000000000000000000..f17ecea1ab5acb9998c057b1d24153d0390ca9fd
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_array_contains-1-d9a90108b052b111e8de4433e008b25a
@@ -0,0 +1 @@
+array_contains(array, value) - Returns TRUE if the array contains value.
diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c b/sql/hive/src/test/resources/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c
deleted file mode 100644
index 0ccae5649af4a9fb2b5054e5eb707df943b422fe..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c
+++ /dev/null
@@ -1,4 +0,0 @@
-array_contains(array, value) - Returns TRUE if the array contains value.
-Example:
-  > SELECT array_contains(array(1, 2, 3), 2) FROM src LIMIT 1;
-  true
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 b/sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247
deleted file mode 100644
index f32a5804e292d30bedf68f62d32fb75d87e99fd9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247
+++ /dev/null
@@ -1 +0,0 @@
-true
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-2-eff16c7836252e01f3d8190cd833f79c b/sql/hive/src/test/resources/golden/udf_array_contains-2-eff16c7836252e01f3d8190cd833f79c
new file mode 100644
index 0000000000000000000000000000000000000000..7f4bbfc04c1c92f689663304c33579d1d1bffe76
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_array_contains-2-eff16c7836252e01f3d8190cd833f79c
@@ -0,0 +1,4 @@
+array_contains(array, value) - Returns TRUE if the array contains value.
+Example:
+  > SELECT array_contains(array(1, 2, 3), 2) FROM src LIMIT 1;
+  true
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c b/sql/hive/src/test/resources/golden/udf_array_contains-3-6b700cb69af298aef45b96bf5ac862d
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c
rename to sql/hive/src/test/resources/golden/udf_array_contains-3-6b700cb69af298aef45b96bf5ac862d
diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 b/sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569
deleted file mode 100644
index f32a5804e292d30bedf68f62d32fb75d87e99fd9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569
+++ /dev/null
@@ -1 +0,0 @@
-true
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d b/sql/hive/src/test/resources/golden/udf_array_contains-4-bdb5a7e0ab81172a438145a1c406b1e8
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d
rename to sql/hive/src/test/resources/golden/udf_array_contains-4-bdb5a7e0ab81172a438145a1c406b1e8
diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udf_ascii-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/udf_ascii-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c b/sql/hive/src/test/resources/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c
deleted file mode 100644
index e121c27212c663125df758fe00199048f159f724..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c
+++ /dev/null
@@ -1 +0,0 @@
-ascii(str) - returns the numeric value of the first character of str
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_ascii-1-72924c23459330cca6a54c70b12a542c b/sql/hive/src/test/resources/golden/udf_ascii-1-72924c23459330cca6a54c70b12a542c
new file mode 100644
index 0000000000000000000000000000000000000000..5e2c4b7209d9beecc6e7018c7c83e6c5c1c436ce
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_ascii-1-72924c23459330cca6a54c70b12a542c
@@ -0,0 +1 @@
+ascii(str) - returns the numeric value of the first character of str
diff --git a/sql/hive/src/test/resources/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 b/sql/hive/src/test/resources/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3
deleted file mode 100644
index 7c9fb835dfe574d00f7dd9c3f345693c7eab2228..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3
+++ /dev/null
@@ -1,6 +0,0 @@
-ascii(str) - returns the numeric value of the first character of str
-Returns 0 if str is empty or NULL if str is NULL
-Example:
-  > SELECT ascii('222') FROM src LIMIT 1;  50
-  > SELECT ascii(2) FROM src LIMIT 1;
-  50
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa b/sql/hive/src/test/resources/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_ascii-2-dab8656d7f001e85615442d60df4b6b3 b/sql/hive/src/test/resources/golden/udf_ascii-2-dab8656d7f001e85615442d60df4b6b3
new file mode 100644
index 0000000000000000000000000000000000000000..87ae6b33499c47d8fbb91f389ea0226c3014450b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_ascii-2-dab8656d7f001e85615442d60df4b6b3
@@ -0,0 +1,6 @@
+ascii(str) - returns the numeric value of the first character of str
+Returns 0 if str is empty or NULL if str is NULL
+Example:
+  > SELECT ascii('222') FROM src LIMIT 1;  50
+  > SELECT ascii(2) FROM src LIMIT 1;
+  50
diff --git a/sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 b/sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1
deleted file mode 100644
index 726fda4c15ed3e73fb166927a6f26714a6bb2376..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1
+++ /dev/null
@@ -1 +0,0 @@
-70	0	33
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e b/sql/hive/src/test/resources/golden/udf_ascii-3-fc25cec86e0dafaf1633c2e3a6d2fc34
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e
rename to sql/hive/src/test/resources/golden/udf_ascii-3-fc25cec86e0dafaf1633c2e3a6d2fc34
diff --git a/sql/hive/src/test/resources/golden/udf_ascii-4-db9a06881a216f0252fa786d98c9bf b/sql/hive/src/test/resources/golden/udf_ascii-4-db9a06881a216f0252fa786d98c9bf
new file mode 100644
index 0000000000000000000000000000000000000000..ba9d9a3b79bab0f97a130efb1d651ce6b2fdf9fc
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_ascii-4-db9a06881a216f0252fa786d98c9bf
@@ -0,0 +1 @@
+70	0	33
diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_asin-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/udf_asin-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd b/sql/hive/src/test/resources/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd
deleted file mode 100644
index 3a56bffc8da34aae375db8739d00bb33455f3357..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd
+++ /dev/null
@@ -1 +0,0 @@
-asin(x) - returns the arc sine of x if -1<=x<=1 or NULL otherwise
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 b/sql/hive/src/test/resources/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129
deleted file mode 100644
index ef207d3f5da38506967e2b0edb025bdb9ee0956c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129
+++ /dev/null
@@ -1,6 +0,0 @@
-asin(x) - returns the arc sine of x if -1<=x<=1 or NULL otherwise
-Example:
-  > SELECT asin(0) FROM src LIMIT 1;
-  0
-  > SELECT asin(2) FROM src LIMIT 1;
-  NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_asin-1-99da197a53767060e3fa0250254d59cd b/sql/hive/src/test/resources/golden/udf_asin-1-99da197a53767060e3fa0250254d59cd
new file mode 100644
index 0000000000000000000000000000000000000000..f5ffb5d0b3939a6509c1ff0845b71bb5469d44cf
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_asin-1-99da197a53767060e3fa0250254d59cd
@@ -0,0 +1 @@
+asin(x) - returns the arc sine of x if -1<=x<=1 or NULL otherwise
diff --git a/sql/hive/src/test/resources/golden/udf_asin-2-3d0c3fa6121f8f5158d221074f1d4129 b/sql/hive/src/test/resources/golden/udf_asin-2-3d0c3fa6121f8f5158d221074f1d4129
new file mode 100644
index 0000000000000000000000000000000000000000..80c16be805ee84926b5c2acd49336f61e1b004f0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_asin-2-3d0c3fa6121f8f5158d221074f1d4129
@@ -0,0 +1,6 @@
+asin(x) - returns the arc sine of x if -1<=x<=1 or NULL otherwise
+Example:
+  > SELECT asin(0) FROM src LIMIT 1;
+  0
+  > SELECT asin(2) FROM src LIMIT 1;
+  NULL
diff --git a/sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f b/sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e b/sql/hive/src/test/resources/golden/udf_asin-3-4b7ee6310a49ebf784a4a712748348ac
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e
rename to sql/hive/src/test/resources/golden/udf_asin-3-4b7ee6310a49ebf784a4a712748348ac
diff --git a/sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 b/sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343
deleted file mode 100644
index 171538eb0b00f4eddffa17929796de55b838f34b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343
+++ /dev/null
@@ -1 +0,0 @@
-0.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 b/sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66
deleted file mode 100644
index a9d9b65be41897853d5aca1ff85eb12d0715a149..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66
+++ /dev/null
@@ -1 +0,0 @@
--0.5235987755982989	0.7208187608700897
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_asin-4-a3edf78ff8ccc629ee7d7518707b69ce b/sql/hive/src/test/resources/golden/udf_asin-4-a3edf78ff8ccc629ee7d7518707b69ce
new file mode 100644
index 0000000000000000000000000000000000000000..ba66466c2a0d0a1080ad71d8e1a67a97f336c384
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_asin-4-a3edf78ff8ccc629ee7d7518707b69ce
@@ -0,0 +1 @@
+0.0
diff --git a/sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 b/sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284
deleted file mode 100644
index b955cad2a39a09a1c4143ccffb078ac94bb80b4c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284
+++ /dev/null
@@ -1 +0,0 @@
-NaN
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_asin-5-8dcbcf784496053e3b57c579aca809a6 b/sql/hive/src/test/resources/golden/udf_asin-5-8dcbcf784496053e3b57c579aca809a6
new file mode 100644
index 0000000000000000000000000000000000000000..46381bfe173d5ba908b025a089a3cbf5491a1c80
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_asin-5-8dcbcf784496053e3b57c579aca809a6
@@ -0,0 +1 @@
+-0.5235987755982989	0.7208187608700897
diff --git a/sql/hive/src/test/resources/golden/udf_asin-6-114c8141f1e831c70d70c570f0ae778f b/sql/hive/src/test/resources/golden/udf_asin-6-114c8141f1e831c70d70c570f0ae778f
new file mode 100644
index 0000000000000000000000000000000000000000..736991a13874574828c7a0fdd2ead5aac88cac65
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_asin-6-114c8141f1e831c70d70c570f0ae778f
@@ -0,0 +1 @@
+NaN
diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udf_atan-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/udf_atan-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b b/sql/hive/src/test/resources/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b
deleted file mode 100644
index 019676ad50bc30d5378a76acb34e8ac9d1ddb028..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b
+++ /dev/null
@@ -1 +0,0 @@
-atan(x) - returns the atan (arctan) of x (x is in radians)
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 b/sql/hive/src/test/resources/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62
deleted file mode 100644
index ce9e52716ff44f796981a03c91952451fa836346..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62
+++ /dev/null
@@ -1,4 +0,0 @@
-atan(x) - returns the atan (arctan) of x (x is in radians)
-Example:
-   > SELECT atan(0) FROM src LIMIT 1;
-  0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_atan-1-c79ed30c2444c8493d0db98c33c9132b b/sql/hive/src/test/resources/golden/udf_atan-1-c79ed30c2444c8493d0db98c33c9132b
new file mode 100644
index 0000000000000000000000000000000000000000..1d34577c5c6b9d91e51b812b0c2d862870d480b4
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_atan-1-c79ed30c2444c8493d0db98c33c9132b
@@ -0,0 +1 @@
+atan(x) - returns the atan (arctan) of x (x is in radians)
diff --git a/sql/hive/src/test/resources/golden/udf_atan-2-77e7ac1b89a4eac9102176cd73f67a62 b/sql/hive/src/test/resources/golden/udf_atan-2-77e7ac1b89a4eac9102176cd73f67a62
new file mode 100644
index 0000000000000000000000000000000000000000..6d7f1d49d9e722ada8e40c8e1aac39b973a918d2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_atan-2-77e7ac1b89a4eac9102176cd73f67a62
@@ -0,0 +1,4 @@
+atan(x) - returns the atan (arctan) of x (x is in radians)
+Example:
+   > SELECT atan(0) FROM src LIMIT 1;
+  0
diff --git a/sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 b/sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 b/sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139
deleted file mode 100644
index 37b3e6018d62c4c8245ab3ca872058ee57507f58..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139
+++ /dev/null
@@ -1 +0,0 @@
-0.7853981633974483	1.4056476493802699	-0.7853981633974483
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e b/sql/hive/src/test/resources/golden/udf_atan-3-e6f97a834028a67e6c3033949f98fbf8
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e
rename to sql/hive/src/test/resources/golden/udf_atan-3-e6f97a834028a67e6c3033949f98fbf8
diff --git a/sql/hive/src/test/resources/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b b/sql/hive/src/test/resources/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b
deleted file mode 100644
index 019676ad50bc30d5378a76acb34e8ac9d1ddb028..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b
+++ /dev/null
@@ -1 +0,0 @@
-atan(x) - returns the atan (arctan) of x (x is in radians)
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_atan-4-eacd47571ba5c67f11e025d8d4de5811 b/sql/hive/src/test/resources/golden/udf_atan-4-eacd47571ba5c67f11e025d8d4de5811
new file mode 100644
index 0000000000000000000000000000000000000000..fe43015585aa4c609a862d9ba51fa078ceb9af38
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_atan-4-eacd47571ba5c67f11e025d8d4de5811
@@ -0,0 +1 @@
+0.7853981633974483	1.4056476493802699	-0.7853981633974483
diff --git a/sql/hive/src/test/resources/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 b/sql/hive/src/test/resources/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62
deleted file mode 100644
index ce9e52716ff44f796981a03c91952451fa836346..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62
+++ /dev/null
@@ -1,4 +0,0 @@
-atan(x) - returns the atan (arctan) of x (x is in radians)
-Example:
-   > SELECT atan(0) FROM src LIMIT 1;
-  0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_atan-5-c79ed30c2444c8493d0db98c33c9132b b/sql/hive/src/test/resources/golden/udf_atan-5-c79ed30c2444c8493d0db98c33c9132b
new file mode 100644
index 0000000000000000000000000000000000000000..1d34577c5c6b9d91e51b812b0c2d862870d480b4
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_atan-5-c79ed30c2444c8493d0db98c33c9132b
@@ -0,0 +1 @@
+atan(x) - returns the atan (arctan) of x (x is in radians)
diff --git a/sql/hive/src/test/resources/golden/udf_atan-6-77e7ac1b89a4eac9102176cd73f67a62 b/sql/hive/src/test/resources/golden/udf_atan-6-77e7ac1b89a4eac9102176cd73f67a62
new file mode 100644
index 0000000000000000000000000000000000000000..6d7f1d49d9e722ada8e40c8e1aac39b973a918d2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_atan-6-77e7ac1b89a4eac9102176cd73f67a62
@@ -0,0 +1,4 @@
+atan(x) - returns the atan (arctan) of x (x is in radians)
+Example:
+   > SELECT atan(0) FROM src LIMIT 1;
+  0
diff --git a/sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 b/sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 b/sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139
deleted file mode 100644
index 37b3e6018d62c4c8245ab3ca872058ee57507f58..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139
+++ /dev/null
@@ -1 +0,0 @@
-0.7853981633974483	1.4056476493802699	-0.7853981633974483
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d b/sql/hive/src/test/resources/golden/udf_atan-7-e6f97a834028a67e6c3033949f98fbf8
similarity index 100%
rename from sql/hive/src/test/resources/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d
rename to sql/hive/src/test/resources/golden/udf_atan-7-e6f97a834028a67e6c3033949f98fbf8
diff --git a/sql/hive/src/test/resources/golden/udf_atan-8-eacd47571ba5c67f11e025d8d4de5811 b/sql/hive/src/test/resources/golden/udf_atan-8-eacd47571ba5c67f11e025d8d4de5811
new file mode 100644
index 0000000000000000000000000000000000000000..fe43015585aa4c609a862d9ba51fa078ceb9af38
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_atan-8-eacd47571ba5c67f11e025d8d4de5811
@@ -0,0 +1 @@
+0.7853981633974483	1.4056476493802699	-0.7853981633974483
diff --git a/sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 b/sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537
index bd171b0e9a29c337f217fb266685b6bce185163e..13ba346a52c935c9c2847b4f7217bf8e991723d5 100644
--- a/sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537
+++ b/sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537
@@ -1 +1 @@
-avg(x) - Returns the mean of a set of numbers
\ No newline at end of file
+avg(x) - Returns the mean of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f b/sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f
index bd171b0e9a29c337f217fb266685b6bce185163e..13ba346a52c935c9c2847b4f7217bf8e991723d5 100644
--- a/sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f
+++ b/sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f
@@ -1 +1 @@
-avg(x) - Returns the mean of a set of numbers
\ No newline at end of file
+avg(x) - Returns the mean of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 b/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927
index 27b6cd42b68cfc57d1b784f05789f136d7bfd003..c538e81220b1f0bd6a9d8591a0932ac6372be0d3 100644
--- a/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927
+++ b/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927
@@ -1 +1 @@
-There is no documentation for function 'bigint'
\ No newline at end of file
+There is no documentation for function 'bigint'
diff --git a/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 b/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202
index 27b6cd42b68cfc57d1b784f05789f136d7bfd003..c538e81220b1f0bd6a9d8591a0932ac6372be0d3 100644
--- a/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202
+++ b/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202
@@ -1 +1 @@
-There is no documentation for function 'bigint'
\ No newline at end of file
+There is no documentation for function 'bigint'
diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udf_bin-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/udf_bin-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b b/sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b
deleted file mode 100644
index 4790a2c1f9292641a34984f973b90c98101831ef..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b
+++ /dev/null
@@ -1 +0,0 @@
-bin(n) - returns n in binary
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 b/sql/hive/src/test/resources/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6
deleted file mode 100644
index 4d82712e858e4a1a8d75534580d40adc869ab0a6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6
+++ /dev/null
@@ -1,5 +0,0 @@
-bin(n) - returns n in binary
-n is a BIGINT. Returns NULL if n is NULL.
-Example:
-  > SELECT bin(13) FROM src LIMIT 1
-  '1101'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_bin-1-ebbb090f6fa6b322a52bec3ba19dfe5b b/sql/hive/src/test/resources/golden/udf_bin-1-ebbb090f6fa6b322a52bec3ba19dfe5b
new file mode 100644
index 0000000000000000000000000000000000000000..ef60184c54fb508639fa21e177bfb73f02d4dc7e
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_bin-1-ebbb090f6fa6b322a52bec3ba19dfe5b
@@ -0,0 +1 @@
+bin(n) - returns n in binary
diff --git a/sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b b/sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b
deleted file mode 100644
index 20fae133f0b4d8e0b1e1a5f02f776cb167d6ef8b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b
+++ /dev/null
@@ -1 +0,0 @@
-1	0	101111101011100001101100101
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_bin-2-843803a1b6ada107c11397af1a2f55d6 b/sql/hive/src/test/resources/golden/udf_bin-2-843803a1b6ada107c11397af1a2f55d6
new file mode 100644
index 0000000000000000000000000000000000000000..115f93fe0fd3e326a6ea353fcc629ed427473be8
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_bin-2-843803a1b6ada107c11397af1a2f55d6
@@ -0,0 +1,5 @@
+bin(n) - returns n in binary
+n is a BIGINT. Returns NULL if n is NULL.
+Example:
+  > SELECT bin(13) FROM src LIMIT 1
+  '1101'
diff --git a/sql/hive/src/test/resources/golden/udf_bin-3-6fda27c8567ac896538cba3f2b230ab b/sql/hive/src/test/resources/golden/udf_bin-3-6fda27c8567ac896538cba3f2b230ab
new file mode 100644
index 0000000000000000000000000000000000000000..83b020539cb9a6ac5975cb6f4813baaee42c6935
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_bin-3-6fda27c8567ac896538cba3f2b230ab
@@ -0,0 +1 @@
+1	0	101111101011100001101100101
diff --git a/sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 b/sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901
deleted file mode 100644
index 76ea1be7f92148d606e213a8b3e95ac6769338e1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901
+++ /dev/null
@@ -1 +0,0 @@
-1111111111111111111111111111111111111111111111111111111111111011
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_bin-4-b9bac215e81c8d5c8324b1287542ced3 b/sql/hive/src/test/resources/golden/udf_bin-4-b9bac215e81c8d5c8324b1287542ced3
new file mode 100644
index 0000000000000000000000000000000000000000..1b12d2f6de47519e012519653a31d00af99df7e7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_bin-4-b9bac215e81c8d5c8324b1287542ced3
@@ -0,0 +1 @@
+1111111111111111111111111111111111111111111111111111111111111011
diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_bitmap_and-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/udf_bitmap_and-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d b/sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d
deleted file mode 100644
index 652a71b48011c31c25a3f65934eb1736a69584dc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d
+++ /dev/null
@@ -1 +0,0 @@
-[13,2,4,8589934592,4096,0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 b/sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25
deleted file mode 100644
index 6d061bf1198c2aab3b30b0210d7f73820c454c5e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25
+++ /dev/null
@@ -1 +0,0 @@
-[13,1,4,2,0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-1-c9e0d8424ec5f433565397b113ae4f57 b/sql/hive/src/test/resources/golden/udf_bitmap_and-1-c9e0d8424ec5f433565397b113ae4f57
new file mode 100644
index 0000000000000000000000000000000000000000..1211192ad4be3a596c6c82ce61c692d6539603c3
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_bitmap_and-1-c9e0d8424ec5f433565397b113ae4f57
@@ -0,0 +1 @@
+[13,2,4,8589934592,4096,0]
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-2-5a60dfc80bef392fa390adddab0c0f87 b/sql/hive/src/test/resources/golden/udf_bitmap_and-2-5a60dfc80bef392fa390adddab0c0f87
new file mode 100644
index 0000000000000000000000000000000000000000..33b9ad33d4f013703098c0e0f9f8e9803fae8b48
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_bitmap_and-2-5a60dfc80bef392fa390adddab0c0f87
@@ -0,0 +1 @@
+[13,1,4,2,0]
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 b/sql/hive/src/test/resources/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 b/sql/hive/src/test/resources/golden/udf_bitmap_and-3-d550d017f9eb9176593719ea34eaae9b
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01
rename to sql/hive/src/test/resources/golden/udf_bitmap_and-3-d550d017f9eb9176593719ea34eaae9b
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 b/sql/hive/src/test/resources/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 b/sql/hive/src/test/resources/golden/udf_bitmap_and-4-a486db1f5a06f9623a0e5abd7737b0c6
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97
rename to sql/hive/src/test/resources/golden/udf_bitmap_and-4-a486db1f5a06f9623a0e5abd7737b0c6
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 b/sql/hive/src/test/resources/golden/udf_bitmap_and-5-549dbeb1293c4c49ae08bf08acdbdf23
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14
rename to sql/hive/src/test/resources/golden/udf_bitmap_and-5-549dbeb1293c4c49ae08bf08acdbdf23
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 b/sql/hive/src/test/resources/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296
deleted file mode 100644
index 43be09952b09cf48a973bf6c3f645aa5043c9aa8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296
+++ /dev/null
@@ -1,10 +0,0 @@
-[13,1,4,2,0]
-[13,1,4,2,0]
-[13,1,4,2,0]
-[13,1,4,2,0]
-[13,1,4,2,0]
-[13,1,4,2,0]
-[13,1,4,2,0]
-[13,1,4,2,0]
-[13,1,4,2,0]
-[13,1,4,2,0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-6-ff2860a163cbe78d5affac8047199296 b/sql/hive/src/test/resources/golden/udf_bitmap_and-6-ff2860a163cbe78d5affac8047199296
new file mode 100644
index 0000000000000000000000000000000000000000..0db773930c274e512cd1f239b3949a3139b6d98a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_bitmap_and-6-ff2860a163cbe78d5affac8047199296
@@ -0,0 +1,10 @@
+[13,1,4,2,0]
+[13,1,4,2,0]
+[13,1,4,2,0]
+[13,1,4,2,0]
+[13,1,4,2,0]
+[13,1,4,2,0]
+[13,1,4,2,0]
+[13,1,4,2,0]
+[13,1,4,2,0]
+[13,1,4,2,0]
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 b/sql/hive/src/test/resources/golden/udf_bitmap_and-7-d550d017f9eb9176593719ea34eaae9b
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72
rename to sql/hive/src/test/resources/golden/udf_bitmap_and-7-d550d017f9eb9176593719ea34eaae9b
diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a
rename to sql/hive/src/test/resources/golden/udf_bitmap_empty-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 b/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90
deleted file mode 100644
index f32a5804e292d30bedf68f62d32fb75d87e99fd9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90
+++ /dev/null
@@ -1 +0,0 @@
-true
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df b/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df
deleted file mode 100644
index 02e4a84d62c4b0fe9cca60bba7b9799f78f1f7ed..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df
+++ /dev/null
@@ -1 +0,0 @@
-false
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad b/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a174269b5d1757398ab7f89cf1c97bfa
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad
rename to sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a174269b5d1757398ab7f89cf1c97bfa
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5 b/sql/hive/src/test/resources/golden/udf_bitmap_empty-2-f5d9880a3278b5632b356bbe6ecd90d3
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5
rename to sql/hive/src/test/resources/golden/udf_bitmap_empty-2-f5d9880a3278b5632b356bbe6ecd90d3
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c b/sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c
deleted file mode 100644
index 652a71b48011c31c25a3f65934eb1736a69584dc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c
+++ /dev/null
@@ -1 +0,0 @@
-[13,2,4,8589934592,4096,0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udf_bitmap_or-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906
rename to sql/hive/src/test/resources/golden/udf_bitmap_or-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef b/sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef
deleted file mode 100644
index 163f4734f4c21524f121e20ded8e946ae20278e5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef
+++ /dev/null
@@ -1 +0,0 @@
-[13,2,4,8589934592,4224,0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-1-cd510a5926df24d1ddbf8d0cce9d76ef b/sql/hive/src/test/resources/golden/udf_bitmap_or-1-cd510a5926df24d1ddbf8d0cce9d76ef
new file mode 100644
index 0000000000000000000000000000000000000000..1211192ad4be3a596c6c82ce61c692d6539603c3
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_bitmap_or-1-cd510a5926df24d1ddbf8d0cce9d76ef
@@ -0,0 +1 @@
+[13,2,4,8589934592,4096,0]
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-2-ab062e08acfd7e552a64ea967a0360c8 b/sql/hive/src/test/resources/golden/udf_bitmap_or-2-ab062e08acfd7e552a64ea967a0360c8
new file mode 100644
index 0000000000000000000000000000000000000000..c1bd6bef82398c6c8bc3b5aa7f7c27bea4a6bed2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_bitmap_or-2-ab062e08acfd7e552a64ea967a0360c8
@@ -0,0 +1 @@
+[13,2,4,8589934592,4224,0]
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 b/sql/hive/src/test/resources/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d b/sql/hive/src/test/resources/golden/udf_bitmap_or-3-d550d017f9eb9176593719ea34eaae9b
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d
rename to sql/hive/src/test/resources/golden/udf_bitmap_or-3-d550d017f9eb9176593719ea34eaae9b
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 b/sql/hive/src/test/resources/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 b/sql/hive/src/test/resources/golden/udf_bitmap_or-4-a486db1f5a06f9623a0e5abd7737b0c6
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30
rename to sql/hive/src/test/resources/golden/udf_bitmap_or-4-a486db1f5a06f9623a0e5abd7737b0c6
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 b/sql/hive/src/test/resources/golden/udf_bitmap_or-5-549dbeb1293c4c49ae08bf08acdbdf23
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7
rename to sql/hive/src/test/resources/golden/udf_bitmap_or-5-549dbeb1293c4c49ae08bf08acdbdf23
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 b/sql/hive/src/test/resources/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0
deleted file mode 100644
index c977a4ce25c3f723d87f6937009294db2db316a1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0
+++ /dev/null
@@ -1,10 +0,0 @@
-[13,2,4,8589934592,4224,0]
-[13,2,4,8589934592,4224,0]
-[13,2,4,8589934592,4224,0]
-[13,2,4,8589934592,4224,0]
-[13,2,4,8589934592,4224,0]
-[13,2,4,8589934592,4224,0]
-[13,2,4,8589934592,4224,0]
-[13,2,4,8589934592,4224,0]
-[13,2,4,8589934592,4224,0]
-[13,2,4,8589934592,4224,0]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-6-ea92fff4d814552b57535ed342a5dde0 b/sql/hive/src/test/resources/golden/udf_bitmap_or-6-ea92fff4d814552b57535ed342a5dde0
new file mode 100644
index 0000000000000000000000000000000000000000..62f4e245e2cddb935bba666476ad8fd9f9cf9e01
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_bitmap_or-6-ea92fff4d814552b57535ed342a5dde0
@@ -0,0 +1,10 @@
+[13,2,4,8589934592,4224,0]
+[13,2,4,8589934592,4224,0]
+[13,2,4,8589934592,4224,0]
+[13,2,4,8589934592,4224,0]
+[13,2,4,8589934592,4224,0]
+[13,2,4,8589934592,4224,0]
+[13,2,4,8589934592,4224,0]
+[13,2,4,8589934592,4224,0]
+[13,2,4,8589934592,4224,0]
+[13,2,4,8589934592,4224,0]
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 b/sql/hive/src/test/resources/golden/udf_bitmap_or-7-d550d017f9eb9176593719ea34eaae9b
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551
rename to sql/hive/src/test/resources/golden/udf_bitmap_or-7-d550d017f9eb9176593719ea34eaae9b
diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a b/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a
index 71cadba7649e884403a2c986de5df799d5357edf..bb00e0889dc07f1c95d6c055c3ae53efd9859fa1 100644
--- a/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a
+++ b/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a
@@ -1 +1 @@
-a & b - Bitwise and
\ No newline at end of file
+a & b - Bitwise and
diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b b/sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b
index ead02cba53aa425660e202347eea3464e5c16ed0..dc67d624d14beb61cfb131328ee53745fb83d615 100644
--- a/sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b
+++ b/sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b
@@ -1,4 +1,4 @@
 a & b - Bitwise and
 Example:
   > SELECT 3 & 5 FROM src LIMIT 1;
-  1
\ No newline at end of file
+  1
diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f b/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f
index 8b06e374c708d45b6d1ba3082574ee32340f54ee..d417f76be994050bfa0dd9f60a6d0b7f1454f3b5 100644
--- a/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f
+++ b/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f
@@ -1 +1 @@
-~ n - Bitwise not
\ No newline at end of file
+~ n - Bitwise not
diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d b/sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d
index 413ca8a653e1fcf316926c92a0d0954cf847c2b3..db249e9032680088eea21f1512f5b05c448ad72d 100644
--- a/sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d
+++ b/sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d
@@ -1,4 +1,4 @@
 ~ n - Bitwise not
 Example:
   > SELECT ~ 0 FROM src LIMIT 1;
-  -1
\ No newline at end of file
+  -1
diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 b/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6
index bdbd28cad97c4e2fc63a782c3fb13d61f05441b6..32343b4afe8bb5f3b7f820adfeaf2a484ced6818 100644
--- a/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6
+++ b/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6
@@ -1 +1 @@
-a | b - Bitwise or
\ No newline at end of file
+a | b - Bitwise or
diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb b/sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb
index 4c116be2acae32458e62e642168de1b563fbd358..8ced93b4f8e88c62cedf40f6c646b7402b0956b4 100644
--- a/sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb
+++ b/sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb
@@ -1,4 +1,4 @@
 a | b - Bitwise or
 Example:
   > SELECT 3 | 5 FROM src LIMIT 1;
-  7
\ No newline at end of file
+  7
diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 b/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16
index b483f75f1b9566abc76751889829dcc2661aa40f..778fda3969951994f189c2dcce8db1fdd109d61a 100644
--- a/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16
+++ b/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16
@@ -1 +1 @@
-a ^ b - Bitwise exclusive or
\ No newline at end of file
+a ^ b - Bitwise exclusive or
diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f b/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f
index 03bf261d7cf51377bd8ab1cbcec58906a3fcadf8..6f1ab55c41ce77cb17663123440abaa63a1483d7 100644
--- a/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f
+++ b/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f
@@ -1,4 +1,4 @@
 a ^ b - Bitwise exclusive or
 Example:
   > SELECT 3 ^ 5 FROM src LIMIT 1;
-  2
\ No newline at end of file
+  2
diff --git a/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 b/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749
index 8134698dfbdcd316ed40958fa4fde2c705d5c711..495b36e68ecdeb581669e75e40b0100f1f8e523d 100644
--- a/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749
+++ b/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749
@@ -1 +1 @@
-There is no documentation for function 'boolean'
\ No newline at end of file
+There is no documentation for function 'boolean'
diff --git a/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e b/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e
index 8134698dfbdcd316ed40958fa4fde2c705d5c711..495b36e68ecdeb581669e75e40b0100f1f8e523d 100644
--- a/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e
+++ b/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e
@@ -1 +1 @@
-There is no documentation for function 'boolean'
\ No newline at end of file
+There is no documentation for function 'boolean'
diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/udf_case-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0
rename to sql/hive/src/test/resources/golden/udf_case-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 b/sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4
deleted file mode 100644
index 645bd7302810f895d7f1e61788a6f8e115b7ac82..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4
+++ /dev/null
@@ -1 +0,0 @@
-There is no documentation for function 'case'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004 b/sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004
deleted file mode 100644
index 645bd7302810f895d7f1e61788a6f8e115b7ac82..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004
+++ /dev/null
@@ -1 +0,0 @@
-There is no documentation for function 'case'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_case-1-5bcbe4c0e0a75462160814a10b9449f4 b/sql/hive/src/test/resources/golden/udf_case-1-5bcbe4c0e0a75462160814a10b9449f4
new file mode 100644
index 0000000000000000000000000000000000000000..e5bc5948a089467def5eb6be6bc1997f45ef0862
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_case-1-5bcbe4c0e0a75462160814a10b9449f4
@@ -0,0 +1 @@
+There is no documentation for function 'case'
diff --git a/sql/hive/src/test/resources/golden/udf_case-2-54acf006155d8822a50e47729be24004 b/sql/hive/src/test/resources/golden/udf_case-2-54acf006155d8822a50e47729be24004
new file mode 100644
index 0000000000000000000000000000000000000000..e5bc5948a089467def5eb6be6bc1997f45ef0862
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_case-2-54acf006155d8822a50e47729be24004
@@ -0,0 +1 @@
+There is no documentation for function 'case'
diff --git a/sql/hive/src/test/resources/golden/udf_case-2-98ee676f92950375917f09d2e492253f b/sql/hive/src/test/resources/golden/udf_case-2-98ee676f92950375917f09d2e492253f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 b/sql/hive/src/test/resources/golden/udf_case-3-48789112d79aeb450d9f49184fc20e1c
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15
rename to sql/hive/src/test/resources/golden/udf_case-3-48789112d79aeb450d9f49184fc20e1c
diff --git a/sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c b/sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c
deleted file mode 100644
index 1ca086d1dca4d37ca303f60803c8010076810019..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c
+++ /dev/null
@@ -1 +0,0 @@
-2	5	15	NULL	20	24
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 b/sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76
deleted file mode 100644
index b920295f69a539ff6e22454082c706636917554f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76
+++ /dev/null
@@ -1 +0,0 @@
-yo
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_case-4-d39ed6ecd256fa99657f13709cb1c6e3 b/sql/hive/src/test/resources/golden/udf_case-4-d39ed6ecd256fa99657f13709cb1c6e3
new file mode 100644
index 0000000000000000000000000000000000000000..a7f5e19894027ef473b40eaecf38aec6fbfbd99f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_case-4-d39ed6ecd256fa99657f13709cb1c6e3
@@ -0,0 +1 @@
+2	5	15	NULL	20	24
diff --git a/sql/hive/src/test/resources/golden/udf_case-5-f53c9bb8a2d20ef7ff1fc7b3403270eb b/sql/hive/src/test/resources/golden/udf_case-5-f53c9bb8a2d20ef7ff1fc7b3403270eb
new file mode 100644
index 0000000000000000000000000000000000000000..092bfb9bdf74dd8cfd22e812151281ee9aa6f01a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_case-5-f53c9bb8a2d20ef7ff1fc7b3403270eb
@@ -0,0 +1 @@
+yo
diff --git a/sql/hive/src/test/resources/golden/udf_case-6-ff583116ba2edd78202349faf1e757dc b/sql/hive/src/test/resources/golden/udf_case-6-ff583116ba2edd78202349faf1e757dc
new file mode 100644
index 0000000000000000000000000000000000000000..db46b5a72aa80f25693fe12224c67ff280cec4cc
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_case-6-ff583116ba2edd78202349faf1e757dc
@@ -0,0 +1 @@
+123	123.0	abcd
diff --git a/sql/hive/src/test/resources/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c b/sql/hive/src/test/resources/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 b/sql/hive/src/test/resources/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 b/sql/hive/src/test/resources/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96
deleted file mode 100644
index d97a0ef52b2e8ace12d09cd0a3a05c0cf70980e1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96
+++ /dev/null
@@ -1,3 +0,0 @@
-1	zero	0
-3	10 is ten	NULL
-100	default	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 b/sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06
index 4854a3c7773acb4ff3ebc249d2ccf1ce159cc25e..badaea42f1d2c8eda2740b9b6425704185f80be5 100644
--- a/sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06
+++ b/sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06
@@ -1 +1 @@
-ceil(x) - Find the smallest integer not smaller than x
\ No newline at end of file
+ceil(x) - Find the smallest integer not smaller than x
diff --git a/sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb b/sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb
index 23685df97892d0e939449a41c69b6ba2277288f4..7d608c5ebd2a3a3afc32c00835f5da63a625cbb5 100644
--- a/sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb
+++ b/sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb
@@ -4,4 +4,4 @@ Example:
   > SELECT ceil(-0.1) FROM src LIMIT 1;
   0
   > SELECT ceil(5) FROM src LIMIT 1;
-  5
\ No newline at end of file
+  5
diff --git a/sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd b/sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd
index bf61b7e89d8b37468b60ec835ec9fa1eb74fc589..31d18aaefac89567ab4238329e5360bd27c22f8d 100644
--- a/sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd
+++ b/sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd
@@ -1 +1 @@
-ceiling(x) - Find the smallest integer not smaller than x
\ No newline at end of file
+ceiling(x) - Find the smallest integer not smaller than x
diff --git a/sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df b/sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df
index 79a06f0ee2a5dc0e9d93675d14a5b32e80a7f485..564d0566997d436050a22d40090edb90e8392433 100644
--- a/sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df
+++ b/sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df
@@ -4,4 +4,4 @@ Example:
   > SELECT ceiling(-0.1) FROM src LIMIT 1;
   0
   > SELECT ceiling(5) FROM src LIMIT 1;
-  5
\ No newline at end of file
+  5
diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 b/sql/hive/src/test/resources/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1
deleted file mode 100644
index 66313c54cc684184f2795680a6ff80dd8bc6a4ed..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1
+++ /dev/null
@@ -1 +0,0 @@
-coalesce(a1, a2, ...) - Returns the first non-null argument
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 b/sql/hive/src/test/resources/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809
deleted file mode 100644
index 04d69977c3df4b36a1be658a6d3ba25e8d64c56e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809
+++ /dev/null
@@ -1,4 +0,0 @@
-coalesce(a1, a2, ...) - Returns the first non-null argument
-Example:
-  > SELECT coalesce(NULL, 1, NULL) FROM src LIMIT 1;
-  1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 b/sql/hive/src/test/resources/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 b/sql/hive/src/test/resources/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2
deleted file mode 100644
index cd5686d89eba4626f06497724591a25333ad6b99..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2
+++ /dev/null
@@ -1 +0,0 @@
-1	1	2	1	3	4	1	1	2	1	3	4	1.0	1.0	2.0	2.0	2.0	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 b/sql/hive/src/test/resources/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 b/sql/hive/src/test/resources/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082
deleted file mode 100644
index 98339f97ef2f8c025c800921435910a3c34b6f6c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082
+++ /dev/null
@@ -1,11 +0,0 @@
-0	0	999
-2	1	999
-4	8	value_2
-6	27	999
-8	64	999
-10	125	999
-12	216	999
-14	343	999
-16	512	999
-18	729	999
-999	999	999
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 b/sql/hive/src/test/resources/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77
deleted file mode 100644
index 59d3f010519033e7cca743c6469ed9fcf985ae09..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77
+++ /dev/null
@@ -1,10 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_CREATEFUNCTION test_udf_get_java_string 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestGetJavaString')
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-
-
diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/udf_concat-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063
rename to sql/hive/src/test/resources/golden/udf_concat-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 b/sql/hive/src/test/resources/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2
deleted file mode 100644
index a3998eeeee623a11185ceda5e93621e7c6a8a00e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2
+++ /dev/null
@@ -1 +0,0 @@
-concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data  bin1, bin2, ... binN
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 b/sql/hive/src/test/resources/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031
deleted file mode 100644
index 0afd542a0503364af7909ab34e29b51bd3ed5076..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031
+++ /dev/null
@@ -1,5 +0,0 @@
-concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data  bin1, bin2, ... binN
-Returns NULL if any argument is NULL.
-Example:
-  > SELECT concat('abc', 'def') FROM src LIMIT 1;
-  'abcdef'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_concat-1-7bc53505a4e6587132870d8d0a704d2 b/sql/hive/src/test/resources/golden/udf_concat-1-7bc53505a4e6587132870d8d0a704d2
new file mode 100644
index 0000000000000000000000000000000000000000..b576089faa4846cba2061daa43cc9427dfce700d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_concat-1-7bc53505a4e6587132870d8d0a704d2
@@ -0,0 +1 @@
+concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data  bin1, bin2, ... binN
diff --git a/sql/hive/src/test/resources/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 b/sql/hive/src/test/resources/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5
deleted file mode 100644
index 69af4e63792dc1fb7ebc4189729db1bb4a783d3d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5
+++ /dev/null
@@ -1 +0,0 @@
-ab	abc	NULL	NULL	a	NULL	123a	12	1	1234abcextra argument
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_concat-2-765c520f239cdff1ea3f8d22ba83e031 b/sql/hive/src/test/resources/golden/udf_concat-2-765c520f239cdff1ea3f8d22ba83e031
new file mode 100644
index 0000000000000000000000000000000000000000..e303d18a0810030d652fcbd357f4219a0679cdc1
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_concat-2-765c520f239cdff1ea3f8d22ba83e031
@@ -0,0 +1,5 @@
+concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data  bin1, bin2, ... binN
+Returns NULL if any argument is NULL.
+Example:
+  > SELECT concat('abc', 'def') FROM src LIMIT 1;
+  'abcdef'
diff --git a/sql/hive/src/test/resources/golden/udf_concat-3-a38183c2685e912befe6246f1b6f93b8 b/sql/hive/src/test/resources/golden/udf_concat-3-a38183c2685e912befe6246f1b6f93b8
new file mode 100644
index 0000000000000000000000000000000000000000..21f03c7abb9b9c4b6e864a762f458e2f0bfa0709
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_concat-3-a38183c2685e912befe6246f1b6f93b8
@@ -0,0 +1 @@
+ab	abc	NULL	NULL	a	NULL	123a	12	1	1234abcextra argument
diff --git a/sql/hive/src/test/resources/golden/udf_concat-4-a20ebbc181e5ee4a1c22ddafd212ddde b/sql/hive/src/test/resources/golden/udf_concat-4-a20ebbc181e5ee4a1c22ddafd212ddde
new file mode 100644
index 0000000000000000000000000000000000000000..09b34f2aa1b69d8567163f692a57b0b4b51d73d4
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_concat-4-a20ebbc181e5ee4a1c22ddafd212ddde
@@ -0,0 +1 @@
+abcd	abcd
diff --git a/sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f
index aecd5d9f2a5c2085a8dc078897507dc4cd701cee..f4cd225e4b2b726e43378aec9f5490e725ac61b1 100644
--- a/sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f
@@ -54,4 +54,4 @@
 1234	95
 1234	96
 1234	97
-1234	98
\ No newline at end of file
+1234	98
diff --git a/sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f
index 455cef3f57162ce2f5066db0d3094516bb5e73f5..fcdbdaa75fd52e52d7b3ec84f0dcd6cdf9f7a2e9 100644
--- a/sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f
+++ b/sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f
@@ -81,4 +81,4 @@
 1234abcextra argument	val_28
 1234abcextra argument	val_37
 1234abcextra argument	val_90
-1234abcextra argument	val_97
\ No newline at end of file
+1234abcextra argument	val_97
diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 b/sql/hive/src/test/resources/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8
deleted file mode 100644
index c901f70500cba61e709042aa8f4e35a6c41127f9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8
+++ /dev/null
@@ -1 +0,0 @@
-concat_ws(separator, [string | array(string)]+) - returns the concatenation of the strings separated by the separator.
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/udf_concat_ws-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/udf_concat_ws-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-1-4c8bd7794a527e544c838408313eeaa8 b/sql/hive/src/test/resources/golden/udf_concat_ws-1-4c8bd7794a527e544c838408313eeaa8
new file mode 100644
index 0000000000000000000000000000000000000000..14d190e03b7552b0fcb947b6951c8f73e0a1b145
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_concat_ws-1-4c8bd7794a527e544c838408313eeaa8
@@ -0,0 +1 @@
+concat_ws(separator, [string | array(string)]+) - returns the concatenation of the strings separated by the separator.
diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab b/sql/hive/src/test/resources/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab
deleted file mode 100644
index 3e751463e055dfd951b9312dd5c5286f021141f1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab
+++ /dev/null
@@ -1,4 +0,0 @@
-concat_ws(separator, [string | array(string)]+) - returns the concatenation of the strings separated by the separator.
-Example:
-  > SELECT concat_ws('.', 'www', array('facebook', 'com')) FROM src LIMIT 1;
-  'www.facebook.com'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-2-b8b80f7e9bf4348367444c73020b3cab b/sql/hive/src/test/resources/golden/udf_concat_ws-2-b8b80f7e9bf4348367444c73020b3cab
new file mode 100644
index 0000000000000000000000000000000000000000..f3be71e54421c744570bd45a6b84daf58fb91295
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_concat_ws-2-b8b80f7e9bf4348367444c73020b3cab
@@ -0,0 +1,4 @@
+concat_ws(separator, [string | array(string)]+) - returns the concatenation of the strings separated by the separator.
+Example:
+  > SELECT concat_ws('.', 'www', array('facebook', 'com')) FROM src LIMIT 1;
+  'www.facebook.com'
diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c b/sql/hive/src/test/resources/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 b/sql/hive/src/test/resources/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e b/sql/hive/src/test/resources/golden/udf_concat_ws-3-ce7c8205cdc107e1fb865d7d48b84a3c
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e
rename to sql/hive/src/test/resources/golden/udf_concat_ws-3-ce7c8205cdc107e1fb865d7d48b84a3c
diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f b/sql/hive/src/test/resources/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 b/sql/hive/src/test/resources/golden/udf_concat_ws-4-b13a1f7f63d2a54efa331c82bd635d63
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952
rename to sql/hive/src/test/resources/golden/udf_concat_ws-4-b13a1f7f63d2a54efa331c82bd635d63
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 b/sql/hive/src/test/resources/golden/udf_concat_ws-5-a507af4996b13433b0ae100fcb32358f
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7
rename to sql/hive/src/test/resources/golden/udf_concat_ws-5-a507af4996b13433b0ae100fcb32358f
diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 b/sql/hive/src/test/resources/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341
deleted file mode 100644
index 7896fd787f3a26387a0b157106aa60c2f679cc42..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341
+++ /dev/null
@@ -1 +0,0 @@
-xyzabc8675309	abc,xyz,8675309	NULL	abc**8675309
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 b/sql/hive/src/test/resources/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-6-ca4f051369183cae36fc9a7bec6a9341 b/sql/hive/src/test/resources/golden/udf_concat_ws-6-ca4f051369183cae36fc9a7bec6a9341
new file mode 100644
index 0000000000000000000000000000000000000000..720cafd9370a7f01f16ce463fb292704447e128a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_concat_ws-6-ca4f051369183cae36fc9a7bec6a9341
@@ -0,0 +1 @@
+xyzabc8675309	abc,xyz,8675309	NULL	abc**8675309
diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c b/sql/hive/src/test/resources/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c
deleted file mode 100644
index a0ec688a3b084dfa56c84a945788fc9756b7217f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c
+++ /dev/null
@@ -1 +0,0 @@
-www.face.book.com.1234	www-face-book-com-1234	wwwFfaceFbookFcomF1234	www_face_book_com_1234	www**face**book**com**1234	www[]face[]book[]com[]1234	wwwAAAfaceAAAbookAAAcomAAA1234
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c b/sql/hive/src/test/resources/golden/udf_concat_ws-7-97071809ba7701b78e3729996f14b591
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c
rename to sql/hive/src/test/resources/golden/udf_concat_ws-7-97071809ba7701b78e3729996f14b591
diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-8-3bfc563ebf7e2cdb811766a54d84f224 b/sql/hive/src/test/resources/golden/udf_concat_ws-8-3bfc563ebf7e2cdb811766a54d84f224
new file mode 100644
index 0000000000000000000000000000000000000000..93b36d28322c37eb26103cc4e28c6fb769b9aa8e
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_concat_ws-8-3bfc563ebf7e2cdb811766a54d84f224
@@ -0,0 +1 @@
+www.face.book.com.1234	www-face-book-com-1234	wwwFfaceFbookFcomF1234	www_face_book_com_1234	www**face**book**com**1234	www[]face[]book[]com[]1234	wwwAAAfaceAAAbookAAAcomAAA1234
diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 b/sql/hive/src/test/resources/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4
deleted file mode 100644
index 0c9f2d12ba117f001e909e0b7b9d0e63bce0c961..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4
+++ /dev/null
@@ -1 +0,0 @@
-NULL	NULL	NULL	NULL	NULL	NULL	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-9-8f60d81b410f4825809aa510806f2df2 b/sql/hive/src/test/resources/golden/udf_concat_ws-9-8f60d81b410f4825809aa510806f2df2
new file mode 100644
index 0000000000000000000000000000000000000000..edb4b1f84001bbeb43d82b6c27d7c53629266e90
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_concat_ws-9-8f60d81b410f4825809aa510806f2df2
@@ -0,0 +1 @@
+NULL	NULL	NULL	NULL	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udf_conv-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66
rename to sql/hive/src/test/resources/golden/udf_conv-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_conv-0-d552befca345f3396464529cfde9f75a b/sql/hive/src/test/resources/golden/udf_conv-0-d552befca345f3396464529cfde9f75a
deleted file mode 100644
index b8fbe88a1997138bead3c610cea3e7a34630c106..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_conv-0-d552befca345f3396464529cfde9f75a
+++ /dev/null
@@ -1 +0,0 @@
-conv(num, from_base, to_base) - convert num from from_base to to_base
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab b/sql/hive/src/test/resources/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab
deleted file mode 100644
index 4f4b0c594c4590af2d0cd43c00d229c6e48412a5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab
+++ /dev/null
@@ -1,7 +0,0 @@
-conv(num, from_base, to_base) - convert num from from_base to to_base
-If to_base is negative, treat num as a signed integer,otherwise, treat it as an unsigned integer.
-Example:
-  > SELECT conv('100', 2, 10) FROM src LIMIT 1;
-  '4'
-  > SELECT conv(-10, 16, -10) FROM src LIMIT 1;
-  '16'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_conv-1-d552befca345f3396464529cfde9f75a b/sql/hive/src/test/resources/golden/udf_conv-1-d552befca345f3396464529cfde9f75a
new file mode 100644
index 0000000000000000000000000000000000000000..0753228c31bd69b4f0d3785c8d1449d61ba0a565
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_conv-1-d552befca345f3396464529cfde9f75a
@@ -0,0 +1 @@
+conv(num, from_base, to_base) - convert num from from_base to to_base
diff --git a/sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 b/sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52
deleted file mode 100644
index 9512cc42415546d9cdff989820c6650c20906535..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52
+++ /dev/null
@@ -1,3 +0,0 @@
-EE	568
-56	134
-137	785
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_conv-10-f9ea15984e84250494e81e25d6a401c0 b/sql/hive/src/test/resources/golden/udf_conv-10-f9ea15984e84250494e81e25d6a401c0
new file mode 100644
index 0000000000000000000000000000000000000000..191900972dd95c2dc19afa83d7a09093a2b03046
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_conv-10-f9ea15984e84250494e81e25d6a401c0
@@ -0,0 +1 @@
+5	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/udf_conv-11-2ce9111b47ed100bebc8d08de54efd1f b/sql/hive/src/test/resources/golden/udf_conv-11-2ce9111b47ed100bebc8d08de54efd1f
new file mode 100644
index 0000000000000000000000000000000000000000..ad3fa0267fa6bc50c4d1d219397d45b0c4d170cc
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_conv-11-2ce9111b47ed100bebc8d08de54efd1f
@@ -0,0 +1,3 @@
+EE	568
+56	134
+137	785
diff --git a/sql/hive/src/test/resources/golden/udf_conv-2-5e5904af10b5d23f20ae28dc4b9a49ab b/sql/hive/src/test/resources/golden/udf_conv-2-5e5904af10b5d23f20ae28dc4b9a49ab
new file mode 100644
index 0000000000000000000000000000000000000000..2d66703d6f3f04e75cdba466983a08d32ffe4ef0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_conv-2-5e5904af10b5d23f20ae28dc4b9a49ab
@@ -0,0 +1,7 @@
+conv(num, from_base, to_base) - convert num from from_base to to_base
+If to_base is negative, treat num as a signed integer,otherwise, treat it as an unsigned integer.
+Example:
+  > SELECT conv('100', 2, 10) FROM src LIMIT 1;
+  '4'
+  > SELECT conv(-10, 16, -10) FROM src LIMIT 1;
+  '16'
diff --git a/sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 b/sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595
deleted file mode 100644
index 91e56a11fa628dd837934c14e93ca0cb2de4f13e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595
+++ /dev/null
@@ -1 +0,0 @@
-3HL	22	33	116ED2B2FB4
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_conv-3-5f43d0bec0421c86c49e2546c5ee923a b/sql/hive/src/test/resources/golden/udf_conv-3-5f43d0bec0421c86c49e2546c5ee923a
new file mode 100644
index 0000000000000000000000000000000000000000..4563fcc478648f18b2d87d6c466d2f0a79380742
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_conv-3-5f43d0bec0421c86c49e2546c5ee923a
@@ -0,0 +1 @@
+3HL	22	33	116ED2B2FB4
diff --git a/sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d b/sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d
deleted file mode 100644
index 08f70d742b8e4e3c0c1212268e49607a62da898b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d
+++ /dev/null
@@ -1 +0,0 @@
--641	B	FFFFFFFFFFFFFFFF	FFFFFFFFFFFFFFF1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55 b/sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55
deleted file mode 100644
index a2a44daa718ccf2368853be9deb1589f32668ff2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55
+++ /dev/null
@@ -1 +0,0 @@
-FFFFFFFFFFFFFFFF	-1	FFFFFFFFFFFFFFFF	-1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_conv-4-5df8d45902a3537e67545e629a96328a b/sql/hive/src/test/resources/golden/udf_conv-4-5df8d45902a3537e67545e629a96328a
new file mode 100644
index 0000000000000000000000000000000000000000..632c3705a0493a63003eade1787015066253b38d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_conv-4-5df8d45902a3537e67545e629a96328a
@@ -0,0 +1 @@
+-641	B	FFFFFFFFFFFFFFFF	FFFFFFFFFFFFFFF1
diff --git a/sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 b/sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183
deleted file mode 100644
index 6948fe3783119ba808ee67c7bc64d62d69d80b03..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183
+++ /dev/null
@@ -1 +0,0 @@
-5	NULL	NULL	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_conv-5-8cdbb45b8c44fa97456da5bc4a43f459 b/sql/hive/src/test/resources/golden/udf_conv-5-8cdbb45b8c44fa97456da5bc4a43f459
new file mode 100644
index 0000000000000000000000000000000000000000..3a7a2ec34f909136a99f0be72742ea16f6ba3eae
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_conv-5-8cdbb45b8c44fa97456da5bc4a43f459
@@ -0,0 +1 @@
+FFFFFFFFFFFFFFFF	-1	FFFFFFFFFFFFFFFF	-1
diff --git a/sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a b/sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a
deleted file mode 100644
index e1021e50fdcbb43e25d9ac0410ab4a96d011964a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a
+++ /dev/null
@@ -1 +0,0 @@
-3HL	22	33
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_conv-6-e5430adfa782ea9094e570d339626c0f b/sql/hive/src/test/resources/golden/udf_conv-6-e5430adfa782ea9094e570d339626c0f
new file mode 100644
index 0000000000000000000000000000000000000000..191900972dd95c2dc19afa83d7a09093a2b03046
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_conv-6-e5430adfa782ea9094e570d339626c0f
@@ -0,0 +1 @@
+5	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/udf_conv-7-687f9b8a09f458d771d5641eec40031b b/sql/hive/src/test/resources/golden/udf_conv-7-687f9b8a09f458d771d5641eec40031b
new file mode 100644
index 0000000000000000000000000000000000000000..c5348e173c243a76a377d9073ca7aaf83c75f63d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_conv-7-687f9b8a09f458d771d5641eec40031b
@@ -0,0 +1 @@
+3HL	22	33
diff --git a/sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e b/sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e
deleted file mode 100644
index 08f70d742b8e4e3c0c1212268e49607a62da898b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e
+++ /dev/null
@@ -1 +0,0 @@
--641	B	FFFFFFFFFFFFFFFF	FFFFFFFFFFFFFFF1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb b/sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb
deleted file mode 100644
index a2a44daa718ccf2368853be9deb1589f32668ff2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb
+++ /dev/null
@@ -1 +0,0 @@
-FFFFFFFFFFFFFFFF	-1	FFFFFFFFFFFFFFFF	-1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_conv-8-384902bf8d45b6b56b2bdc5db550c10 b/sql/hive/src/test/resources/golden/udf_conv-8-384902bf8d45b6b56b2bdc5db550c10
new file mode 100644
index 0000000000000000000000000000000000000000..632c3705a0493a63003eade1787015066253b38d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_conv-8-384902bf8d45b6b56b2bdc5db550c10
@@ -0,0 +1 @@
+-641	B	FFFFFFFFFFFFFFFF	FFFFFFFFFFFFFFF1
diff --git a/sql/hive/src/test/resources/golden/udf_conv-9-28b833d0cd96f74c23aa7cf8c4f5a167 b/sql/hive/src/test/resources/golden/udf_conv-9-28b833d0cd96f74c23aa7cf8c4f5a167
new file mode 100644
index 0000000000000000000000000000000000000000..3a7a2ec34f909136a99f0be72742ea16f6ba3eae
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_conv-9-28b833d0cd96f74c23aa7cf8c4f5a167
@@ -0,0 +1 @@
+FFFFFFFFFFFFFFFF	-1	FFFFFFFFFFFFFFFF	-1
diff --git a/sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 b/sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957
deleted file mode 100644
index 6948fe3783119ba808ee67c7bc64d62d69d80b03..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957
+++ /dev/null
@@ -1 +0,0 @@
-5	NULL	NULL	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 b/sql/hive/src/test/resources/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3
deleted file mode 100644
index 99a7d42bb5b57eade99d4b95666b899ca48ad1ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3
+++ /dev/null
@@ -1 +0,0 @@
-cos(x) - returns the cosine of x (x is in radians)
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/udf_cos-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063
rename to sql/hive/src/test/resources/golden/udf_cos-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 b/sql/hive/src/test/resources/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5
deleted file mode 100644
index 196294de1f19f1ebb2f20d3934e08e784c7d19f6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5
+++ /dev/null
@@ -1,4 +0,0 @@
-cos(x) - returns the cosine of x (x is in radians)
-Example:
-   > SELECT cos(0) FROM src LIMIT 1;
-  1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_cos-1-44f411146a7190b89f2bc8b4aa61cae3 b/sql/hive/src/test/resources/golden/udf_cos-1-44f411146a7190b89f2bc8b4aa61cae3
new file mode 100644
index 0000000000000000000000000000000000000000..fa0e6975503cc14e63ccfdf79820600f0440098f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_cos-1-44f411146a7190b89f2bc8b4aa61cae3
@@ -0,0 +1 @@
+cos(x) - returns the cosine of x (x is in radians)
diff --git a/sql/hive/src/test/resources/golden/udf_cos-2-176030bdf43ff83ed8b3112d0c79f2f5 b/sql/hive/src/test/resources/golden/udf_cos-2-176030bdf43ff83ed8b3112d0c79f2f5
new file mode 100644
index 0000000000000000000000000000000000000000..09d61507338022ba593bfb3a50e80c7fb78148de
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_cos-2-176030bdf43ff83ed8b3112d0c79f2f5
@@ -0,0 +1,4 @@
+cos(x) - returns the cosine of x (x is in radians)
+Example:
+   > SELECT cos(0) FROM src LIMIT 1;
+  1
diff --git a/sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 b/sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 b/sql/hive/src/test/resources/golden/udf_cos-3-166acc86afd6ececfe43800e38f106c9
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312
rename to sql/hive/src/test/resources/golden/udf_cos-3-166acc86afd6ececfe43800e38f106c9
diff --git a/sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 b/sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429
deleted file mode 100644
index c0690acff887bf9ca73fc5f916ad1f4cbd9181ae..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429
+++ /dev/null
@@ -1 +0,0 @@
-0.5570225467662173	7.963267107332633E-4	0.8775825618903728
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_cos-4-efda2f85872c6144280970eab84ef4d4 b/sql/hive/src/test/resources/golden/udf_cos-4-efda2f85872c6144280970eab84ef4d4
new file mode 100644
index 0000000000000000000000000000000000000000..14417ab71b694263144477b570472ec0f9eb1e4a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_cos-4-efda2f85872c6144280970eab84ef4d4
@@ -0,0 +1 @@
+0.5570225467662173	7.963267107332633E-4	0.8775825618903728
diff --git a/sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560 b/sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560
index e01715295adc8b328ea7321267b50d90d8a4fbfa..56cf7bdf7b040b725581292a30dfe6bd278bfd17 100644
--- a/sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560
+++ b/sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560
@@ -1,3 +1,3 @@
 count(*) - Returns the total number of retrieved rows, including rows containing NULL values.
 count(expr) - Returns the number of rows for which the supplied expression is non-NULL.
-count(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL.
\ No newline at end of file
+count(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL.
diff --git a/sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f b/sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f
index e01715295adc8b328ea7321267b50d90d8a4fbfa..56cf7bdf7b040b725581292a30dfe6bd278bfd17 100644
--- a/sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f
+++ b/sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f
@@ -1,3 +1,3 @@
 count(*) - Returns the total number of retrieved rows, including rows containing NULL values.
 count(expr) - Returns the number of rows for which the supplied expression is non-NULL.
-count(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL.
\ No newline at end of file
+count(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL.
diff --git a/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e b/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e
+++ b/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e b/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e
+++ b/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 b/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7
+++ b/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 b/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8
+++ b/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 b/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4
index e0da8ae09ae60f2173f5ab88e31a7359c855066a..7536e3d32619277adaee6ca2925c00ef2dedde54 100644
--- a/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4
+++ b/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4
@@ -1 +1 @@
-309
\ No newline at end of file
+309
diff --git a/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 b/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6
index e0da8ae09ae60f2173f5ab88e31a7359c855066a..7536e3d32619277adaee6ca2925c00ef2dedde54 100644
--- a/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6
+++ b/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6
@@ -1 +1 @@
-309
\ No newline at end of file
+309
diff --git a/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 b/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643
index eb1f49486af7c892e115f610fa32b505125766fc..1b79f38e25b24dcac0318f3371793a6ec204a71b 100644
--- a/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643
+++ b/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643
@@ -1 +1 @@
-500
\ No newline at end of file
+500
diff --git a/sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 b/sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45
index 83e0fc4e63a250398b2796bf19631ed35a4fa942..2e77bafd12f7dbbf89f2608297037634d1331920 100644
--- a/sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45
+++ b/sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45
@@ -1 +1 @@
-date_add(start_date, num_days) - Returns the date that is num_days after start_date.
\ No newline at end of file
+date_add(start_date, num_days) - Returns the date that is num_days after start_date.
diff --git a/sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 b/sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2
index 83b9851499f710c634a9d9ab9e00831c0cb60c05..3c91e138d7bd58b07ec943044c137d055398052b 100644
--- a/sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2
+++ b/sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2
@@ -2,4 +2,4 @@ date_add(start_date, num_days) - Returns the date that is num_days after start_d
 start_date is a string in the format 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'. num_days is a number. The time part of start_date is ignored.
 Example:
    > SELECT date_add('2009-30-07', 1) FROM src LIMIT 1;
-  '2009-31-07'
\ No newline at end of file
+  '2009-31-07'
diff --git a/sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 b/sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33
index 7dec81f2b6d4e3344e67d22792f7ab4c478db78a..3ee6ac48208522613f943fe60e9a72b07f7199bc 100644
--- a/sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33
+++ b/sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33
@@ -1 +1 @@
-date_sub(start_date, num_days) - Returns the date that is num_days before start_date.
\ No newline at end of file
+date_sub(start_date, num_days) - Returns the date that is num_days before start_date.
diff --git a/sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 b/sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4
index 105b63424062b7427aa135a8a96e9dff94a0d4f6..29d663f35c5861b8707690d80d9e896628c0b1b1 100644
--- a/sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4
+++ b/sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4
@@ -2,4 +2,4 @@ date_sub(start_date, num_days) - Returns the date that is num_days before start_
 start_date is a string in the format 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'. num_days is a number. The time part of start_date is ignored.
 Example:
    > SELECT date_sub('2009-30-07', 1) FROM src LIMIT 1;
-  '2009-29-07'
\ No newline at end of file
+  '2009-29-07'
diff --git a/sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 b/sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5
index b59d456397d535b53391f65a7bedb369a777ff18..64cae647c80052c8bbe6250e8417044aed8128ab 100644
--- a/sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5
+++ b/sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5
@@ -1 +1 @@
-datediff(date1, date2) - Returns the number of days between date1 and date2
\ No newline at end of file
+datediff(date1, date2) - Returns the number of days between date1 and date2
diff --git a/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 b/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5
index c240df94594d490faadfb2152a1e56328b61641f..7ccaee7ad3bd478bd2cf2567e570a19bbf185256 100644
--- a/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5
+++ b/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5
@@ -2,4 +2,4 @@ datediff(date1, date2) - Returns the number of days between date1 and date2
 date1 and date2 are strings in the format 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'. The time parts are ignored.If date1 is earlier than date2, the result is negative.
 Example:
    > SELECT datediff('2009-30-07', '2009-31-07') FROM src LIMIT 1;
-  1
\ No newline at end of file
+  1
diff --git a/sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 b/sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756
index 11e32a4509b3f801a51375347f75f2934dca1185..d4017178b4e6b50862dc8db50c1cf573ed4d7f9e 100644
--- a/sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756
+++ b/sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756
@@ -1 +1 @@
-day(date) - Returns the date of the month of date
\ No newline at end of file
+day(date) - Returns the date of the month of date
diff --git a/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4 b/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4
index 9da0858ba92a567afc20c35e632bd7e39dcd652d..6135aafa50860491f95b356b5b1b8378b64fe45d 100644
--- a/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4
+++ b/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4
@@ -3,4 +3,4 @@ Synonyms: dayofmonth
 date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'.
 Example:
    > SELECT day('2009-30-07', 1) FROM src LIMIT 1;
-  30
\ No newline at end of file
+  30
diff --git a/sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 b/sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502
index 33e90a2af295ff953fd8b17f07aea6ab1176462a..47a7018d9d5acd47b3aaf9a6f3ef2e260c8a03cf 100644
--- a/sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502
+++ b/sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502
@@ -1 +1 @@
-dayofmonth(date) - Returns the date of the month of date
\ No newline at end of file
+dayofmonth(date) - Returns the date of the month of date
diff --git a/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 b/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8
index ee9911af3248cfcfe4f18d8766a0edfc649bb08b..d9490e20a3b6dbb95600cac6ffb2ebb36ab28150 100644
--- a/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8
+++ b/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8
@@ -3,4 +3,4 @@ Synonyms: day
 date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'.
 Example:
    > SELECT dayofmonth('2009-30-07', 1) FROM src LIMIT 1;
-  30
\ No newline at end of file
+  30
diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/udf_degrees-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/udf_degrees-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae b/sql/hive/src/test/resources/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_degrees-1-f24ce67606944e23a4adc79f91cf0c17
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b
rename to sql/hive/src/test/resources/golden/udf_degrees-1-f24ce67606944e23a4adc79f91cf0c17
diff --git a/sql/hive/src/test/resources/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 b/sql/hive/src/test/resources/golden/udf_degrees-2-aaee5dd4e87eaae3e65a585e07f1a3e4
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7
rename to sql/hive/src/test/resources/golden/udf_degrees-2-aaee5dd4e87eaae3e65a585e07f1a3e4
diff --git a/sql/hive/src/test/resources/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b b/sql/hive/src/test/resources/golden/udf_degrees-3-2fd3a55901fe765f8f154531a7f5fd6b
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b
rename to sql/hive/src/test/resources/golden/udf_degrees-3-2fd3a55901fe765f8f154531a7f5fd6b
diff --git a/sql/hive/src/test/resources/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 b/sql/hive/src/test/resources/golden/udf_degrees-4-42f653c3c3cc3c94bb9ab9c5a4d1ca96
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96
rename to sql/hive/src/test/resources/golden/udf_degrees-4-42f653c3c3cc3c94bb9ab9c5a4d1ca96
diff --git a/sql/hive/src/test/resources/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae b/sql/hive/src/test/resources/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_degrees-5-3a6468b02be2605c91b31987e76fb9a8
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b
rename to sql/hive/src/test/resources/golden/udf_degrees-5-3a6468b02be2605c91b31987e76fb9a8
diff --git a/sql/hive/src/test/resources/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 b/sql/hive/src/test/resources/golden/udf_degrees-6-aaee5dd4e87eaae3e65a585e07f1a3e4
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7
rename to sql/hive/src/test/resources/golden/udf_degrees-6-aaee5dd4e87eaae3e65a585e07f1a3e4
diff --git a/sql/hive/src/test/resources/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b b/sql/hive/src/test/resources/golden/udf_degrees-7-2fd3a55901fe765f8f154531a7f5fd6b
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b
rename to sql/hive/src/test/resources/golden/udf_degrees-7-2fd3a55901fe765f8f154531a7f5fd6b
diff --git a/sql/hive/src/test/resources/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 b/sql/hive/src/test/resources/golden/udf_degrees-8-42f653c3c3cc3c94bb9ab9c5a4d1ca96
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96
rename to sql/hive/src/test/resources/golden/udf_degrees-8-42f653c3c3cc3c94bb9ab9c5a4d1ca96
diff --git a/sql/hive/src/test/resources/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db b/sql/hive/src/test/resources/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db
deleted file mode 100644
index d98718752a36f736ed0a52182fd8f766bd3ff509..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db
+++ /dev/null
@@ -1 +0,0 @@
-a div b - Divide a by b rounded to the long integer
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e b/sql/hive/src/test/resources/golden/udf_div-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e
rename to sql/hive/src/test/resources/golden/udf_div-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_div-1-31d31c7d5c544327dabfd874c88314db b/sql/hive/src/test/resources/golden/udf_div-1-31d31c7d5c544327dabfd874c88314db
new file mode 100644
index 0000000000000000000000000000000000000000..b02a7003ce76817a25b18ac554bc609530be958e
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_div-1-31d31c7d5c544327dabfd874c88314db
@@ -0,0 +1 @@
+a div b - Divide a by b rounded to the long integer
diff --git a/sql/hive/src/test/resources/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 b/sql/hive/src/test/resources/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8
deleted file mode 100644
index 59265a74f9b3f802fa2ec9c9f4f9c32f3eb9b2d0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8
+++ /dev/null
@@ -1,4 +0,0 @@
-a div b - Divide a by b rounded to the long integer
-Example:
-  > SELECT 3 div 2 FROM src LIMIT 1;
-  1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b b/sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b
deleted file mode 100644
index 56a6051ca2b02b04ef92d5150c9ef600403cb1de..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b
+++ /dev/null
@@ -1 +0,0 @@
-1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_div-2-f23a07b1c6b1a98b303863188c10a8d8 b/sql/hive/src/test/resources/golden/udf_div-2-f23a07b1c6b1a98b303863188c10a8d8
new file mode 100644
index 0000000000000000000000000000000000000000..6b79eff6e60924755b16cc9d6df595dd77709ab4
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_div-2-f23a07b1c6b1a98b303863188c10a8d8
@@ -0,0 +1,4 @@
+a div b - Divide a by b rounded to the long integer
+Example:
+  > SELECT 3 div 2 FROM src LIMIT 1;
+  1
diff --git a/sql/hive/src/test/resources/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 b/sql/hive/src/test/resources/golden/udf_div-3-5111340caad64e36370d9d4bc4db5f27
similarity index 100%
rename from sql/hive/src/test/resources/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5
rename to sql/hive/src/test/resources/golden/udf_div-3-5111340caad64e36370d9d4bc4db5f27
diff --git a/sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 b/sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208
deleted file mode 100644
index 8b623e47785f64c394442295cb730bc09e24415d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208
+++ /dev/null
@@ -1 +0,0 @@
-a / b - Divide a by b
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 b/sql/hive/src/test/resources/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840
deleted file mode 100644
index 2acf2b6a64648f66095f629c3ba8ed475259139e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840
+++ /dev/null
@@ -1,4 +0,0 @@
-a / b - Divide a by b
-Example:
-  > SELECT 3 / 2 FROM src LIMIT 1;
-  1.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 b/sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50
deleted file mode 100644
index 400122e60f5997d465a46a63eaeeb15cb7dbfcdf..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50
+++ /dev/null
@@ -1 +0,0 @@
-1.5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 b/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617
index 54de9e9f8edecd8dd0ced05e77f4a332d7ddb459..1d0a61f1b635f63828b8747c11c975e9953778bf 100644
--- a/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617
+++ b/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617
@@ -1 +1 @@
-There is no documentation for function 'double'
\ No newline at end of file
+There is no documentation for function 'double'
diff --git a/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 b/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4
index 54de9e9f8edecd8dd0ced05e77f4a332d7ddb459..1d0a61f1b635f63828b8747c11c975e9953778bf 100644
--- a/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4
+++ b/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4
@@ -1 +1 @@
-There is no documentation for function 'double'
\ No newline at end of file
+There is no documentation for function 'double'
diff --git a/sql/hive/src/test/resources/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 b/sql/hive/src/test/resources/golden/udf_elt-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71
rename to sql/hive/src/test/resources/golden/udf_elt-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f b/sql/hive/src/test/resources/golden/udf_elt-1-b46b060da76d1772db998c26a62a608f
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f
rename to sql/hive/src/test/resources/golden/udf_elt-1-b46b060da76d1772db998c26a62a608f
diff --git a/sql/hive/src/test/resources/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a b/sql/hive/src/test/resources/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 b/sql/hive/src/test/resources/golden/udf_elt-2-e9f1bf17ad2a6f7bf3e40798ceebdbf4
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4
rename to sql/hive/src/test/resources/golden/udf_elt-2-e9f1bf17ad2a6f7bf3e40798ceebdbf4
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_elt-3-c2554fac72a2a51bb33faae16aec3507
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b
rename to sql/hive/src/test/resources/golden/udf_elt-3-c2554fac72a2a51bb33faae16aec3507
diff --git a/sql/hive/src/test/resources/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 b/sql/hive/src/test/resources/golden/udf_elt-4-533ad9c703c320a6556c09dd5f9ac351
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814
rename to sql/hive/src/test/resources/golden/udf_elt-4-533ad9c703c320a6556c09dd5f9ac351
diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/sql/hive/src/test/resources/golden/udf_equal-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0
rename to sql/hive/src/test/resources/golden/udf_equal-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_equal-0-36b6cdf7c5f68c91155569b1622f5876 b/sql/hive/src/test/resources/golden/udf_equal-1-36b6cdf7c5f68c91155569b1622f5876
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_equal-0-36b6cdf7c5f68c91155569b1622f5876
rename to sql/hive/src/test/resources/golden/udf_equal-1-36b6cdf7c5f68c91155569b1622f5876
diff --git a/sql/hive/src/test/resources/golden/udf_equal-1-2422b50b96502dde8b661acdfebd8892 b/sql/hive/src/test/resources/golden/udf_equal-2-2422b50b96502dde8b661acdfebd8892
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_equal-1-2422b50b96502dde8b661acdfebd8892
rename to sql/hive/src/test/resources/golden/udf_equal-2-2422b50b96502dde8b661acdfebd8892
diff --git a/sql/hive/src/test/resources/golden/udf_equal-2-e0faab0f5e736c24bcc5503aeac55053 b/sql/hive/src/test/resources/golden/udf_equal-3-e0faab0f5e736c24bcc5503aeac55053
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_equal-2-e0faab0f5e736c24bcc5503aeac55053
rename to sql/hive/src/test/resources/golden/udf_equal-3-e0faab0f5e736c24bcc5503aeac55053
diff --git a/sql/hive/src/test/resources/golden/udf_equal-3-39d8d6f197803de927f0af5409ec2f33 b/sql/hive/src/test/resources/golden/udf_equal-4-39d8d6f197803de927f0af5409ec2f33
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_equal-3-39d8d6f197803de927f0af5409ec2f33
rename to sql/hive/src/test/resources/golden/udf_equal-4-39d8d6f197803de927f0af5409ec2f33
diff --git a/sql/hive/src/test/resources/golden/udf_equal-4-94ac2476006425e1b3bcddf29ad07b16 b/sql/hive/src/test/resources/golden/udf_equal-5-ee018fc3267dbdd55b60ed4e6f56c9ca
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_equal-4-94ac2476006425e1b3bcddf29ad07b16
rename to sql/hive/src/test/resources/golden/udf_equal-5-ee018fc3267dbdd55b60ed4e6f56c9ca
diff --git a/sql/hive/src/test/resources/golden/udf_equal-5-878650cf21e9360a07d204c8ffb0cde7 b/sql/hive/src/test/resources/golden/udf_equal-6-878650cf21e9360a07d204c8ffb0cde7
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_equal-5-878650cf21e9360a07d204c8ffb0cde7
rename to sql/hive/src/test/resources/golden/udf_equal-6-878650cf21e9360a07d204c8ffb0cde7
diff --git a/sql/hive/src/test/resources/golden/udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e b/sql/hive/src/test/resources/golden/udf_equal-7-1635ef051fecdfc7891d9f5a9a3a545e
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e
rename to sql/hive/src/test/resources/golden/udf_equal-7-1635ef051fecdfc7891d9f5a9a3a545e
diff --git a/sql/hive/src/test/resources/golden/udf_equal-7-78f1b96c199e307714fa1b804e5bae27 b/sql/hive/src/test/resources/golden/udf_equal-8-276101b04b10b7cd6d59061a8cbf42d2
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_equal-7-78f1b96c199e307714fa1b804e5bae27
rename to sql/hive/src/test/resources/golden/udf_equal-8-276101b04b10b7cd6d59061a8cbf42d2
diff --git a/sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a b/sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a
index 9b3dbf316d1fbb0760e89c75dc460c8f2508dece..2a5080be93ac47fde948423672045393023e03fe 100644
--- a/sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a
+++ b/sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a
@@ -1 +1 @@
-exp(x) - Returns e to the power of x
\ No newline at end of file
+exp(x) - Returns e to the power of x
diff --git a/sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 b/sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6
index a42c95bb8d213c445d23b540a6a8fcf2810b1ee5..b5a4d037f4014f296fed881a737efb3889231d96 100644
--- a/sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6
+++ b/sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6
@@ -1,4 +1,4 @@
 exp(x) - Returns e to the power of x
 Example:
    > SELECT exp(0) FROM src LIMIT 1;
-  1
\ No newline at end of file
+  1
diff --git a/sql/hive/src/test/resources/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 b/sql/hive/src/test/resources/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33
deleted file mode 100644
index a30bc26f5ba589f147764711df4643a96f8bf907..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33
+++ /dev/null
@@ -1 +0,0 @@
-field(str, str1, str2, ...) - returns the index of str in the str1,str2,... list or 0 if not found
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/udf_field-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0
rename to sql/hive/src/test/resources/golden/udf_field-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_field-1-277b4a4dcb38cabb6df50147c77e0a33 b/sql/hive/src/test/resources/golden/udf_field-1-277b4a4dcb38cabb6df50147c77e0a33
new file mode 100644
index 0000000000000000000000000000000000000000..2e6133785ac7cea1ffe044470dde4d78c3ceb2b5
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_field-1-277b4a4dcb38cabb6df50147c77e0a33
@@ -0,0 +1 @@
+field(str, str1, str2, ...) - returns the index of str in the str1,str2,... list or 0 if not found
diff --git a/sql/hive/src/test/resources/golden/udf_field-1-379d8580693493f620a8f4084709324f b/sql/hive/src/test/resources/golden/udf_field-1-379d8580693493f620a8f4084709324f
deleted file mode 100644
index bb55c0a1db1f501bdde6aee2218bd7f2e581aac3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_field-1-379d8580693493f620a8f4084709324f
+++ /dev/null
@@ -1,2 +0,0 @@
-field(str, str1, str2, ...) - returns the index of str in the str1,str2,... list or 0 if not found
-All primitive types are supported, arguments are compared using str.equals(x). If str is NULL, the return value is 0.
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 b/sql/hive/src/test/resources/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8
deleted file mode 100644
index 275f46482425dea7d0038c6a275fed3e5fd0ce69..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8
+++ /dev/null
@@ -1,2 +0,0 @@
-86	val_86	0	0	2	1	1	0	0
-66	val_66	1	1	0	0	0	0	0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_field-10-ca9db7e6bb687606bc273d1f6c191035
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b
rename to sql/hive/src/test/resources/golden/udf_field-10-ca9db7e6bb687606bc273d1f6c191035
diff --git a/sql/hive/src/test/resources/golden/udf_field-11-7982ea72163dbc4cd45f53454edf66c8 b/sql/hive/src/test/resources/golden/udf_field-11-7982ea72163dbc4cd45f53454edf66c8
new file mode 100644
index 0000000000000000000000000000000000000000..a13456f1bfdda662837ccf5ea46cd477b775df8f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_field-11-7982ea72163dbc4cd45f53454edf66c8
@@ -0,0 +1,2 @@
+86	val_86	0	0	2	1	1	0	0
+66	val_66	1	1	0	0	0	0	0
diff --git a/sql/hive/src/test/resources/golden/udf_field-2-379d8580693493f620a8f4084709324f b/sql/hive/src/test/resources/golden/udf_field-2-379d8580693493f620a8f4084709324f
new file mode 100644
index 0000000000000000000000000000000000000000..f9d418fee7b537ccb6780401064cf9a77d3a2394
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_field-2-379d8580693493f620a8f4084709324f
@@ -0,0 +1,2 @@
+field(str, str1, str2, ...) - returns the index of str in the str1,str2,... list or 0 if not found
+All primitive types are supported, arguments are compared using str.equals(x). If str is NULL, the return value is 0.
diff --git a/sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b b/sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b
deleted file mode 100644
index c42823854fb0f83578a05510b09c89760b7eb325..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b
+++ /dev/null
@@ -1 +0,0 @@
-0	0	0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/div-0-5e7fc5719c7265bc1d6af52005ebad03 b/sql/hive/src/test/resources/golden/udf_field-3-5960d42257b272f6ba043406229bbf26
similarity index 100%
rename from sql/hive/src/test/resources/golden/div-0-5e7fc5719c7265bc1d6af52005ebad03
rename to sql/hive/src/test/resources/golden/udf_field-3-5960d42257b272f6ba043406229bbf26
diff --git a/sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 b/sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98
deleted file mode 100644
index 58692342498081ccee5c9f73f64260f5bfe9672d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98
+++ /dev/null
@@ -1 +0,0 @@
-1	2	3	4	4
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_field-4-212d8b2297bf6a3311d24d68de67b5c6 b/sql/hive/src/test/resources/golden/udf_field-4-212d8b2297bf6a3311d24d68de67b5c6
new file mode 100644
index 0000000000000000000000000000000000000000..e5449f0bfa4732e49f717753e8599625cd49c0bf
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_field-4-212d8b2297bf6a3311d24d68de67b5c6
@@ -0,0 +1 @@
+1	2	3	4	4
diff --git a/sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 b/sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29
deleted file mode 100644
index 58692342498081ccee5c9f73f64260f5bfe9672d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29
+++ /dev/null
@@ -1 +0,0 @@
-1	2	3	4	4
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 b/sql/hive/src/test/resources/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_field-5-949c2de113b214d83734c0c177f04b6b b/sql/hive/src/test/resources/golden/udf_field-5-949c2de113b214d83734c0c177f04b6b
new file mode 100644
index 0000000000000000000000000000000000000000..e5449f0bfa4732e49f717753e8599625cd49c0bf
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_field-5-949c2de113b214d83734c0c177f04b6b
@@ -0,0 +1 @@
+1	2	3	4	4
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_field-6-2d7c5cbe891c4a9dda34f9842f8e0828
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b
rename to sql/hive/src/test/resources/golden/udf_field-6-2d7c5cbe891c4a9dda34f9842f8e0828
diff --git a/sql/hive/src/test/resources/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 b/sql/hive/src/test/resources/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_field-7-3fd8b0c333acdf28c676315b03e2e10
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b
rename to sql/hive/src/test/resources/golden/udf_field-7-3fd8b0c333acdf28c676315b03e2e10
diff --git a/sql/hive/src/test/resources/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef b/sql/hive/src/test/resources/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef
deleted file mode 100644
index a76563207da24a481946534c9e097a42338f6b5f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef
+++ /dev/null
@@ -1,2 +0,0 @@
-86	val_86	0	0	2	0	0	0	0	0	2	0
-66	val_66	1	1	0	0	0	1	0	0	2	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 b/sql/hive/src/test/resources/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_field-8-6aa3518e9f55299754521e959e9376ef b/sql/hive/src/test/resources/golden/udf_field-8-6aa3518e9f55299754521e959e9376ef
new file mode 100644
index 0000000000000000000000000000000000000000..9af9d61b8b1352d92e361be765b34cd1b5f6a3e5
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_field-8-6aa3518e9f55299754521e959e9376ef
@@ -0,0 +1,2 @@
+86	val_86	0	0	2	0	0	0	0	0	2	0
+66	val_66	1	1	0	0	0	1	0	0	2	2
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 b/sql/hive/src/test/resources/golden/udf_field-9-66dc6c81db0ac9b2075783b0d8976083
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64
rename to sql/hive/src/test/resources/golden/udf_field-9-66dc6c81db0ac9b2075783b0d8976083
diff --git a/sql/hive/src/test/resources/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 b/sql/hive/src/test/resources/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 b/sql/hive/src/test/resources/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2
deleted file mode 100644
index f14679978b79e14e2e9403cc7396b1981526483b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2
+++ /dev/null
@@ -1 +0,0 @@
-find_in_set(str,str_array) - Returns the first occurrence  of str in str_array where str_array is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument has any commas.
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/udf_find_in_set-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063
rename to sql/hive/src/test/resources/golden/udf_find_in_set-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-1-18d3e88b18c18a00598146a3307276f2 b/sql/hive/src/test/resources/golden/udf_find_in_set-1-18d3e88b18c18a00598146a3307276f2
new file mode 100644
index 0000000000000000000000000000000000000000..342fb7fad55b0997ac4dec8b979e46d8492710d7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_find_in_set-1-18d3e88b18c18a00598146a3307276f2
@@ -0,0 +1 @@
+find_in_set(str,str_array) - Returns the first occurrence  of str in str_array where str_array is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument has any commas.
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 b/sql/hive/src/test/resources/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246
deleted file mode 100644
index 1bebc68416bff4557f26b2d787cde7ca3fe249ab..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246
+++ /dev/null
@@ -1,7 +0,0 @@
-find_in_set(str,str_array) - Returns the first occurrence  of str in str_array where str_array is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument has any commas.
-Example:
-  > SELECT find_in_set('ab','abc,b,ab,c,def') FROM src LIMIT 1;
-  3
-  > SELECT * FROM src1 WHERE NOT find_in_set(key,'311,128,345,956')=0;
-  311  val_311
-  128
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c b/sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c
deleted file mode 100644
index 56a6051ca2b02b04ef92d5150c9ef600403cb1de..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c
+++ /dev/null
@@ -1 +0,0 @@
-1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87 b/sql/hive/src/test/resources/golden/udf_find_in_set-10-df21f44247d7275a292520c1605c4aab
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87
rename to sql/hive/src/test/resources/golden/udf_find_in_set-10-df21f44247d7275a292520c1605c4aab
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a b/sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a
deleted file mode 100644
index 56a6051ca2b02b04ef92d5150c9ef600403cb1de..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a
+++ /dev/null
@@ -1 +0,0 @@
-1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e b/sql/hive/src/test/resources/golden/udf_find_in_set-11-91f8c37820f31d0d1b16029a59a185ad
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e
rename to sql/hive/src/test/resources/golden/udf_find_in_set-11-91f8c37820f31d0d1b16029a59a185ad
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 b/sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3
deleted file mode 100644
index d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3
+++ /dev/null
@@ -1 +0,0 @@
-2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 b/sql/hive/src/test/resources/golden/udf_find_in_set-12-692f41c998bbc6bec0f374eae4356739
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298
rename to sql/hive/src/test/resources/golden/udf_find_in_set-12-692f41c998bbc6bec0f374eae4356739
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a b/sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a
deleted file mode 100644
index e440e5c842586965a7fb77deda2eca68612b1f53..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a
+++ /dev/null
@@ -1 +0,0 @@
-3
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-13-45e5ae8f60de2c41f189db7922a04917 b/sql/hive/src/test/resources/golden/udf_find_in_set-13-45e5ae8f60de2c41f189db7922a04917
new file mode 100644
index 0000000000000000000000000000000000000000..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_find_in_set-13-45e5ae8f60de2c41f189db7922a04917
@@ -0,0 +1 @@
+2
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 b/sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/decimal_2-25-14face5c7104382196e65741a199c36 b/sql/hive/src/test/resources/golden/udf_find_in_set-14-8e410ecfad2d408ad7d2554ccd3a6621
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_2-25-14face5c7104382196e65741a199c36
rename to sql/hive/src/test/resources/golden/udf_find_in_set-14-8e410ecfad2d408ad7d2554ccd3a6621
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb b/sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/udf_find_in_set-15-c50e6ff95c05bfa854b33b03db858cd9
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817
rename to sql/hive/src/test/resources/golden/udf_find_in_set-15-c50e6ff95c05bfa854b33b03db858cd9
diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/sql/hive/src/test/resources/golden/udf_find_in_set-16-8e17f41ae6e8b1075af4790a8fd88e13
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0
rename to sql/hive/src/test/resources/golden/udf_find_in_set-16-8e17f41ae6e8b1075af4790a8fd88e13
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 b/sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 b/sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8
deleted file mode 100644
index 99f516951ae7dd12f52612cdc70fa7f33dbef948..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8
+++ /dev/null
@@ -1,2 +0,0 @@
-311	val_311
-128	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udf_find_in_set-17-fe61f992f2d971d006155bdec3143803
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66
rename to sql/hive/src/test/resources/golden/udf_find_in_set-17-fe61f992f2d971d006155bdec3143803
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-18-5b556a29e1685605bcc47bce60cf66c8 b/sql/hive/src/test/resources/golden/udf_find_in_set-18-5b556a29e1685605bcc47bce60cf66c8
new file mode 100644
index 0000000000000000000000000000000000000000..01228944b05a53836dbad7469bc91b1bf372083a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_find_in_set-18-5b556a29e1685605bcc47bce60cf66c8
@@ -0,0 +1,2 @@
+311	val_311
+128	
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-2-5fb7a7a1725749dc3853eb80fba19246 b/sql/hive/src/test/resources/golden/udf_find_in_set-2-5fb7a7a1725749dc3853eb80fba19246
new file mode 100644
index 0000000000000000000000000000000000000000..d856144af1f86a5c80365e22285d52992893f521
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_find_in_set-2-5fb7a7a1725749dc3853eb80fba19246
@@ -0,0 +1,7 @@
+find_in_set(str,str_array) - Returns the first occurrence  of str in str_array where str_array is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument has any commas.
+Example:
+  > SELECT find_in_set('ab','abc,b,ab,c,def') FROM src LIMIT 1;
+  3
+  > SELECT * FROM src1 WHERE NOT find_in_set(key,'311,128,345,956')=0;
+  311  val_311
+  128
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 b/sql/hive/src/test/resources/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e b/sql/hive/src/test/resources/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e
deleted file mode 100644
index 5817cbcff62acb57fe91b3ee9ed30b004647163a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e
+++ /dev/null
@@ -1,25 +0,0 @@
-1
-NULL
-1
-NULL
-NULL
-NULL
-1
-1
-1
-NULL
-NULL
-NULL
-1
-1
-1
-1
-1
-1
-1
-1
-1
-1
-NULL
-NULL
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb b/sql/hive/src/test/resources/golden/udf_find_in_set-3-b3823bdc04a7f98951b55c3e30d2a772
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb
rename to sql/hive/src/test/resources/golden/udf_find_in_set-3-b3823bdc04a7f98951b55c3e30d2a772
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-4-132b7bc7812db7683eb3bff607275d0e b/sql/hive/src/test/resources/golden/udf_find_in_set-4-132b7bc7812db7683eb3bff607275d0e
new file mode 100644
index 0000000000000000000000000000000000000000..df725cb2c6aa7af781c0bcdfdd1609c7f5a8f2d5
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_find_in_set-4-132b7bc7812db7683eb3bff607275d0e
@@ -0,0 +1,25 @@
+1
+NULL
+1
+NULL
+NULL
+NULL
+1
+1
+1
+NULL
+NULL
+NULL
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+NULL
+NULL
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b b/sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b
deleted file mode 100644
index 56a6051ca2b02b04ef92d5150c9ef600403cb1de..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b
+++ /dev/null
@@ -1 +0,0 @@
-1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 b/sql/hive/src/test/resources/golden/udf_find_in_set-5-6f25b5bba89e1fcae171f5d595acc4ee
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055
rename to sql/hive/src/test/resources/golden/udf_find_in_set-5-6f25b5bba89e1fcae171f5d595acc4ee
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f b/sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f
deleted file mode 100644
index d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f
+++ /dev/null
@@ -1 +0,0 @@
-2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 b/sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823
deleted file mode 100644
index e440e5c842586965a7fb77deda2eca68612b1f53..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823
+++ /dev/null
@@ -1 +0,0 @@
-3
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-6-7bf387b94afb755faca4ad73bb7c42ba b/sql/hive/src/test/resources/golden/udf_find_in_set-6-7bf387b94afb755faca4ad73bb7c42ba
new file mode 100644
index 0000000000000000000000000000000000000000..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_find_in_set-6-7bf387b94afb755faca4ad73bb7c42ba
@@ -0,0 +1 @@
+2
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 b/sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 b/sql/hive/src/test/resources/golden/udf_find_in_set-7-730d5e95ef748ad946eceefbcd633826
similarity index 100%
rename from sql/hive/src/test/resources/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5
rename to sql/hive/src/test/resources/golden/udf_find_in_set-7-730d5e95ef748ad946eceefbcd633826
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 b/sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/udf_find_in_set-8-ea11724531f191940e455d13878a0e69
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063
rename to sql/hive/src/test/resources/golden/udf_find_in_set-8-ea11724531f191940e455d13878a0e69
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343 b/sql/hive/src/test/resources/golden/udf_find_in_set-9-81f9999ed1b063ce7f17d89bd0162777
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343
rename to sql/hive/src/test/resources/golden/udf_find_in_set-9-81f9999ed1b063ce7f17d89bd0162777
diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 b/sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 b/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7
index 9d15b5f5956d75669c3eb703e131fda0cb0cc89b..467e25bc261c2b1b7784c8d51c5cc81493a8e1af 100644
--- a/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7
+++ b/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7
@@ -1 +1 @@
-There is no documentation for function 'float'
\ No newline at end of file
+There is no documentation for function 'float'
diff --git a/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b b/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b
index 9d15b5f5956d75669c3eb703e131fda0cb0cc89b..467e25bc261c2b1b7784c8d51c5cc81493a8e1af 100644
--- a/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b
+++ b/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b
@@ -1 +1 @@
-There is no documentation for function 'float'
\ No newline at end of file
+There is no documentation for function 'float'
diff --git a/sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 b/sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6
index c76710bfcc1b5934a3ccd2215ca33e5c85a45b3a..de1563b40b83613c9cdd2a77dcdf3c61b8be95e5 100644
--- a/sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6
+++ b/sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6
@@ -1 +1 @@
-floor(x) - Find the largest integer not greater than x
\ No newline at end of file
+floor(x) - Find the largest integer not greater than x
diff --git a/sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 b/sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248
index 3f73eea16e1836c8d59956a5bde8e62777be12ec..ab6951202eb2a2c440f2de7afadab08458278615 100644
--- a/sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248
+++ b/sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248
@@ -3,4 +3,4 @@ Example:
   > SELECT floor(-0.1) FROM src LIMIT 1;
   -1
   > SELECT floor(5) FROM src LIMIT 1;
-  5
\ No newline at end of file
+  5
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/udf_format_number-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0
rename to sql/hive/src/test/resources/golden/udf_format_number-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 b/sql/hive/src/test/resources/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5
deleted file mode 100644
index c981e1726c0704a0999855cac7844d8c1988b01c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5
+++ /dev/null
@@ -1 +0,0 @@
-format_number(X, D) - Formats the number X to a format like '#,###,###.##', rounded to D decimal places, and returns the result as a string. If D is 0, the result has no decimal point or fractional part. This is supposed to function like MySQL's FORMAT
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb b/sql/hive/src/test/resources/golden/udf_format_number-1-e86d559aeb84a4cc017a103182c22bfb
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb
rename to sql/hive/src/test/resources/golden/udf_format_number-1-e86d559aeb84a4cc017a103182c22bfb
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-10-3bddca6913ea7e281e223b0603010b77 b/sql/hive/src/test/resources/golden/udf_format_number-10-3bddca6913ea7e281e223b0603010b77
new file mode 100644
index 0000000000000000000000000000000000000000..8ee27a864b6ddfe27dd190031efbf2016ef3bca0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_format_number-10-3bddca6913ea7e281e223b0603010b77
@@ -0,0 +1 @@
+-9,223,372,036,854,775,807.0000000000	9,223,372,036,854,775,807.00000000000000000000	0.000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000005	179,769,313,486,231,570,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-2-525f133cfff86d44afdeeda667c365a5 b/sql/hive/src/test/resources/golden/udf_format_number-2-525f133cfff86d44afdeeda667c365a5
new file mode 100644
index 0000000000000000000000000000000000000000..14a40602519b821b60dd1f5933f147e966016b2a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_format_number-2-525f133cfff86d44afdeeda667c365a5
@@ -0,0 +1 @@
+format_number(X, D) - Formats the number X to a format like '#,###,###.##', rounded to D decimal places, and returns the result as a string. If D is 0, the result has no decimal point or fractional part. This is supposed to function like MySQL's FORMAT
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd b/sql/hive/src/test/resources/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd
deleted file mode 100644
index b2aa527ca0ddba67b553bf1f701c29685105372c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd
+++ /dev/null
@@ -1,4 +0,0 @@
-format_number(X, D) - Formats the number X to a format like '#,###,###.##', rounded to D decimal places, and returns the result as a string. If D is 0, the result has no decimal point or fractional part. This is supposed to function like MySQL's FORMAT
-Example:
-  > SELECT format_number(12332.123456, 4) FROM src LIMIT 1;
-  '12,332.1235'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-3-591f302d5c1cd24e153a598fa0b352fd b/sql/hive/src/test/resources/golden/udf_format_number-3-591f302d5c1cd24e153a598fa0b352fd
new file mode 100644
index 0000000000000000000000000000000000000000..def95a79e7375ab76add8d840c8704818dfa67cc
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_format_number-3-591f302d5c1cd24e153a598fa0b352fd
@@ -0,0 +1,4 @@
+format_number(X, D) - Formats the number X to a format like '#,###,###.##', rounded to D decimal places, and returns the result as a string. If D is 0, the result has no decimal point or fractional part. This is supposed to function like MySQL's FORMAT
+Example:
+  > SELECT format_number(12332.123456, 4) FROM src LIMIT 1;
+  '12,332.1235'
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c b/sql/hive/src/test/resources/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 b/sql/hive/src/test/resources/golden/udf_format_number-4-295d41a2146a27320c2be90499343260
deleted file mode 100644
index 89e118cc62bf3eb111ed467ce83bdb4d766e07e9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_format_number-4-295d41a2146a27320c2be90499343260
+++ /dev/null
@@ -1 +0,0 @@
-12,332.1235	12,332.1000	12,332
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb b/sql/hive/src/test/resources/golden/udf_format_number-4-7969ffc4e80f7214a8eead8e1084368a
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb
rename to sql/hive/src/test/resources/golden/udf_format_number-4-7969ffc4e80f7214a8eead8e1084368a
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 b/sql/hive/src/test/resources/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9
deleted file mode 100644
index 33e21fa7dbfc4b545703f99b41da830bd34e704b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9
+++ /dev/null
@@ -1 +0,0 @@
-0.123456789000	12,345,678.12346	1,234,567.1234568	123,456
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-5-da5bf00d45d2bd758489f661caffd4dc b/sql/hive/src/test/resources/golden/udf_format_number-5-da5bf00d45d2bd758489f661caffd4dc
new file mode 100644
index 0000000000000000000000000000000000000000..055b84b4b35d8bc1da3f4f8039cfc05f7b7bea02
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_format_number-5-da5bf00d45d2bd758489f661caffd4dc
@@ -0,0 +1 @@
+12,332.1235	12,332.1000	12,332
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-6-6dfca21d142652fec9017ba828a226c8 b/sql/hive/src/test/resources/golden/udf_format_number-6-6dfca21d142652fec9017ba828a226c8
new file mode 100644
index 0000000000000000000000000000000000000000..9f12f9921318d6dd3678d9cc40bd842ac734089c
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_format_number-6-6dfca21d142652fec9017ba828a226c8
@@ -0,0 +1 @@
+0.123456789000	12,345,678.12346	1,234,567.1234568	123,456
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 b/sql/hive/src/test/resources/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3
deleted file mode 100644
index 07b05cd16f5a340704897492aa0ee7d062ac30a8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3
+++ /dev/null
@@ -1 +0,0 @@
--123,456	-1,234,567.12	-0.123456789000000	-12,345.1235
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-7-31eda4b0f31406add3a61e2503590113 b/sql/hive/src/test/resources/golden/udf_format_number-7-31eda4b0f31406add3a61e2503590113
new file mode 100644
index 0000000000000000000000000000000000000000..032768d6889430bcc1e221cabd1ab2f0e8cd64f1
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_format_number-7-31eda4b0f31406add3a61e2503590113
@@ -0,0 +1 @@
+-123,456	-1,234,567.12	-0.123456789000000	-12,345.1235
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 b/sql/hive/src/test/resources/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766
deleted file mode 100644
index 2b600a6a83aa8f393f68a54a9d5dc77073cfd76e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766
+++ /dev/null
@@ -1 +0,0 @@
-0.0000	0.0	0.0	0.0	-0.0000
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-8-b297476c6348209933132202030eb8ea b/sql/hive/src/test/resources/golden/udf_format_number-8-b297476c6348209933132202030eb8ea
new file mode 100644
index 0000000000000000000000000000000000000000..8077e5a60e4ef57b3eaebd66194ad657404ad2a9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_format_number-8-b297476c6348209933132202030eb8ea
@@ -0,0 +1 @@
+0.0000	0.0	0.0	0.0	-0.0000
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e b/sql/hive/src/test/resources/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e
deleted file mode 100644
index d027b2cb0b94aa39305770e52998e0f282f62974..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e
+++ /dev/null
@@ -1 +0,0 @@
-0	1.0000	12.00	123.00000	1,234.0000000
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 b/sql/hive/src/test/resources/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646
deleted file mode 100644
index afdec63c8dfcaeee929c10a4c8ebf2e8ec7720ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646
+++ /dev/null
@@ -1 +0,0 @@
--9,223,372,036,854,775,807.0000000000	9,223,372,036,854,775,807.00000000000000000000	0.000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000005	179,769,313,486,231,570,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_format_number-9-a21fbe58ff475634c8ed9829c6b8c187 b/sql/hive/src/test/resources/golden/udf_format_number-9-a21fbe58ff475634c8ed9829c6b8c187
new file mode 100644
index 0000000000000000000000000000000000000000..f9f98b94234f39b342093626bed7a6380d7fe8ec
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_format_number-9-a21fbe58ff475634c8ed9829c6b8c187
@@ -0,0 +1 @@
+0	1.0000	12.00	123.00000	1,234.0000000
diff --git a/sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f b/sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f
index 10ce106f121ba3e689a2d6271e9f0572be8dfe88..38550ea4196254e47e891c54fa8f0a6ab68d1953 100644
--- a/sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f
+++ b/sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f
@@ -1 +1 @@
-from_unixtime(unix_time, format) - returns unix_time in the specified format
\ No newline at end of file
+from_unixtime(unix_time, format) - returns unix_time in the specified format
diff --git a/sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 b/sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65
index 20f4d2b35c49acb3b25002029240acbedbe1270b..ef15f822d80f52ba2c00782154f18a79db05807c 100644
--- a/sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65
+++ b/sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65
@@ -1,4 +1,4 @@
 from_unixtime(unix_time, format) - returns unix_time in the specified format
 Example:
   > SELECT from_unixtime(0, 'yyyy-MM-dd HH:mm:ss') FROM src LIMIT 1;
-  '1970-01-01 00:00:00'
\ No newline at end of file
+  '1970-01-01 00:00:00'
diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f b/sql/hive/src/test/resources/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f
deleted file mode 100644
index 4e4f3f7255fd37833f93f262c84262f9c53ade48..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f
+++ /dev/null
@@ -1 +0,0 @@
-get_json_object(json_txt, path) - Extract a json object from path 
diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 b/sql/hive/src/test/resources/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917
deleted file mode 100644
index 0e1dcf934a0239585f07b75595af7fa344caf6a4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917
+++ /dev/null
@@ -1,16 +0,0 @@
-get_json_object(json_txt, path) - Extract a json object from path 
-Extract json object from a json string based on json path specified, and return json string of the extracted json object. It will return null if the input json string is invalid.
-A limited version of JSONPath supported:
-  $   : Root object
-  .   : Child operator
-  []  : Subscript operator for array
-  *   : Wildcard for []
-Syntax not supported that's worth noticing:
-  ''  : Zero length string as key
-  ..  : Recursive descent
-  &amp;#064;   : Current object/element
-  ()  : Script expression
-  ?() : Filter (script) expression.
-  [,] : Union operator
-  [start:end:step] : array slice operator
-
diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 b/sql/hive/src/test/resources/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 b/sql/hive/src/test/resources/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/udf_greaterthan-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717
rename to sql/hive/src/test/resources/golden/udf_greaterthan-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 b/sql/hive/src/test/resources/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4
deleted file mode 100644
index 54d6096d07bbe6622eea6e48f2ecf295aaaf99c3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4
+++ /dev/null
@@ -1 +0,0 @@
-a > b - Returns TRUE if a is greater than b
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c b/sql/hive/src/test/resources/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c
deleted file mode 100644
index 54d6096d07bbe6622eea6e48f2ecf295aaaf99c3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c
+++ /dev/null
@@ -1 +0,0 @@
-a > b - Returns TRUE if a is greater than b
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-1-99d268829a124103cb3429c53fdc4de4 b/sql/hive/src/test/resources/golden/udf_greaterthan-1-99d268829a124103cb3429c53fdc4de4
new file mode 100644
index 0000000000000000000000000000000000000000..1eec522da2a1e48bb74248454410583b6274f8f7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_greaterthan-1-99d268829a124103cb3429c53fdc4de4
@@ -0,0 +1 @@
+a > b - Returns TRUE if a is greater than b
diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f b/sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f
deleted file mode 100644
index 679b0376125f03c0fd59aa1ef6ac78c8e3d79b07..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f
+++ /dev/null
@@ -1 +0,0 @@
-true	false	false	false
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-2-8aab8e39726a986e10e1e572939fd63c b/sql/hive/src/test/resources/golden/udf_greaterthan-2-8aab8e39726a986e10e1e572939fd63c
new file mode 100644
index 0000000000000000000000000000000000000000..1eec522da2a1e48bb74248454410583b6274f8f7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_greaterthan-2-8aab8e39726a986e10e1e572939fd63c
@@ -0,0 +1 @@
+a > b - Returns TRUE if a is greater than b
diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-3-75fcadcdc6c050f1c7e70c71dc89c800 b/sql/hive/src/test/resources/golden/udf_greaterthan-3-75fcadcdc6c050f1c7e70c71dc89c800
new file mode 100644
index 0000000000000000000000000000000000000000..096c64e2afd933b6c245434676068ce7a04d503c
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_greaterthan-3-75fcadcdc6c050f1c7e70c71dc89c800
@@ -0,0 +1 @@
+true	false	false	false
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66
rename to sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a
deleted file mode 100644
index abf7dfdab730cdd0b87feccbae24b7a64c15b01e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a
+++ /dev/null
@@ -1 +0,0 @@
-a >= b - Returns TRUE if a is not smaller than b
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451
deleted file mode 100644
index abf7dfdab730cdd0b87feccbae24b7a64c15b01e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451
+++ /dev/null
@@ -1 +0,0 @@
-a >= b - Returns TRUE if a is not smaller than b
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-a7214027a91abf6501881e2af313347a b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-a7214027a91abf6501881e2af313347a
new file mode 100644
index 0000000000000000000000000000000000000000..d5422146acd0e59f0326a644bed495b0164ae400
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-a7214027a91abf6501881e2af313347a
@@ -0,0 +1 @@
+a >= b - Returns TRUE if a is not smaller than b
diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-3669f2008e7f428f365efadbcb5ae451 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-3669f2008e7f428f365efadbcb5ae451
new file mode 100644
index 0000000000000000000000000000000000000000..d5422146acd0e59f0326a644bed495b0164ae400
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-3669f2008e7f428f365efadbcb5ae451
@@ -0,0 +1 @@
+a >= b - Returns TRUE if a is not smaller than b
diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819
deleted file mode 100644
index 1fb1894fc21f4c62842adecaa3677a6587a6aaab..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819
+++ /dev/null
@@ -1 +0,0 @@
-true	false	true	true
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-3-631662997e0c8d24d80b5d64a17446d2 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-3-631662997e0c8d24d80b5d64a17446d2
new file mode 100644
index 0000000000000000000000000000000000000000..435a5f7b0efb306272409a3c89aec5da0e1a3a95
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-3-631662997e0c8d24d80b5d64a17446d2
@@ -0,0 +1 @@
+true	false	true	true
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/udf_hash-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147
rename to sql/hive/src/test/resources/golden/udf_hash-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 b/sql/hive/src/test/resources/golden/udf_hash-1-b9e3a3986320d275982797140edfccf4
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4
rename to sql/hive/src/test/resources/golden/udf_hash-1-b9e3a3986320d275982797140edfccf4
diff --git a/sql/hive/src/test/resources/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a b/sql/hive/src/test/resources/golden/udf_hash-2-a18646b51501d0b1beb967dc79afbd1a
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a
rename to sql/hive/src/test/resources/golden/udf_hash-2-a18646b51501d0b1beb967dc79afbd1a
diff --git a/sql/hive/src/test/resources/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 b/sql/hive/src/test/resources/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 b/sql/hive/src/test/resources/golden/udf_hash-3-2646a87ce26c383a9dafea9b56281ab7
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72
rename to sql/hive/src/test/resources/golden/udf_hash-3-2646a87ce26c383a9dafea9b56281ab7
diff --git a/sql/hive/src/test/resources/golden/udf_hash-3-23991312391d518aacf3d4469c816eae b/sql/hive/src/test/resources/golden/udf_hash-4-d1368c2e3cd113e46202156b44811987
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_hash-3-23991312391d518aacf3d4469c816eae
rename to sql/hive/src/test/resources/golden/udf_hash-4-d1368c2e3cd113e46202156b44811987
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/udf_hex-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634
rename to sql/hive/src/test/resources/golden/udf_hex-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 b/sql/hive/src/test/resources/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16
deleted file mode 100644
index f87a6117eacf7bf4010ece3124d40177310de6fc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16
+++ /dev/null
@@ -1 +0,0 @@
-hex(n, bin, or str) - Convert the argument to hexadecimal 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_hex-1-c8b923c23d5eb31446780f28acbd4e16 b/sql/hive/src/test/resources/golden/udf_hex-1-c8b923c23d5eb31446780f28acbd4e16
new file mode 100644
index 0000000000000000000000000000000000000000..c45cea8bc13b8a4324e077355f748dd935bc6966
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_hex-1-c8b923c23d5eb31446780f28acbd4e16
@@ -0,0 +1 @@
+hex(n, bin, or str) - Convert the argument to hexadecimal 
diff --git a/sql/hive/src/test/resources/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c b/sql/hive/src/test/resources/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c
deleted file mode 100644
index e8cee4afc3a80f3ff05adce20a914488db703d51..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c
+++ /dev/null
@@ -1,8 +0,0 @@
-hex(n, bin, or str) - Convert the argument to hexadecimal 
-If the argument is a string, returns two hex digits for each character in the string.
-If the argument is a number or binary, returns the hexadecimal representation.
-Example:
-  > SELECT hex(17) FROM src LIMIT 1;
-  'H1'
-  > SELECT hex('Facebook') FROM src LIMIT 1;
-  '46616365626F6F6B'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 b/sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559
deleted file mode 100644
index 34eb75a6c784a3f190eaad6b411b5eab2a060398..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559
+++ /dev/null
@@ -1 +0,0 @@
-46616365626F6F6B	00	71776572747975696F706173646667686A6B6C
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_hex-2-d55348c0ccd133b7abb690f6949b520c b/sql/hive/src/test/resources/golden/udf_hex-2-d55348c0ccd133b7abb690f6949b520c
new file mode 100644
index 0000000000000000000000000000000000000000..bcc46336d0fd5608dacd0807cccd4b04579a5385
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_hex-2-d55348c0ccd133b7abb690f6949b520c
@@ -0,0 +1,8 @@
+hex(n, bin, or str) - Convert the argument to hexadecimal 
+If the argument is a string, returns two hex digits for each character in the string.
+If the argument is a number or binary, returns the hexadecimal representation.
+Example:
+  > SELECT hex(17) FROM src LIMIT 1;
+  'H1'
+  > SELECT hex('Facebook') FROM src LIMIT 1;
+  '46616365626F6F6B'
diff --git a/sql/hive/src/test/resources/golden/udf_hex-3-3a1de5be8ce350612ee6a93303107470 b/sql/hive/src/test/resources/golden/udf_hex-3-3a1de5be8ce350612ee6a93303107470
new file mode 100644
index 0000000000000000000000000000000000000000..b0ffe57c8e16125828c263b518e3ff5c201fc174
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_hex-3-3a1de5be8ce350612ee6a93303107470
@@ -0,0 +1 @@
+46616365626F6F6B	00	71776572747975696F706173646667686A6B6C
diff --git a/sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf b/sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf
deleted file mode 100644
index 2a45250c67a633bbef87e98136202a035bf1b702..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf
+++ /dev/null
@@ -1 +0,0 @@
-1	0	FACEB005
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_hex-4-a7f99c9ad67c837658b924c0a979cf01 b/sql/hive/src/test/resources/golden/udf_hex-4-a7f99c9ad67c837658b924c0a979cf01
new file mode 100644
index 0000000000000000000000000000000000000000..8c56faa0f9db95cd249e7b92d06f863137d9381b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_hex-4-a7f99c9ad67c837658b924c0a979cf01
@@ -0,0 +1 @@
+1	0	FACEB005
diff --git a/sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea b/sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea
deleted file mode 100644
index 50d9557967ac436fc9c36085fbded1da6ee7b7c9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea
+++ /dev/null
@@ -1 +0,0 @@
-FFFFFFFFFFFFFFFB
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_hex-5-1a9b53bd38a3693e66f7c03092e15c8e b/sql/hive/src/test/resources/golden/udf_hex-5-1a9b53bd38a3693e66f7c03092e15c8e
new file mode 100644
index 0000000000000000000000000000000000000000..b766160c677047add56444d12637a3da8d8fd98b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_hex-5-1a9b53bd38a3693e66f7c03092e15c8e
@@ -0,0 +1 @@
+FFFFFFFFFFFFFFFB
diff --git a/sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce b/sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce
deleted file mode 100644
index 48911456dc33917e08dd4fcc70dc08e68431fbbe..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce
+++ /dev/null
@@ -1 +0,0 @@
-hour(date) - Returns the hour of date
diff --git a/sql/hive/src/test/resources/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 b/sql/hive/src/test/resources/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04
deleted file mode 100644
index be1a966635202d1ef11690b23c13e031a61e3dca..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04
+++ /dev/null
@@ -1,7 +0,0 @@
-hour(date) - Returns the hour of date
-date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'.
-Example:
-   > SELECT hour('2009-07-30 12:58:59') FROM src LIMIT 1;
-  12
-  > SELECT hour('12:58:59') FROM src LIMIT 1;
-  12
diff --git a/sql/hive/src/test/resources/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a b/sql/hive/src/test/resources/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 b/sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613
deleted file mode 100644
index 1a35f9f158133ccc9e1ce7fa7d07bf9181a0b5dd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613
+++ /dev/null
@@ -1 +0,0 @@
-13	13	NULL
diff --git a/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a b/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a
deleted file mode 100644
index b1bd38b62a0800a4f6a80c34e21c5acffae52c7e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a
+++ /dev/null
@@ -1 +0,0 @@
-13
diff --git a/sql/hive/src/test/resources/golden/input16_cc-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/udf_if-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/input16_cc-0-43d53504df013e6b35f81811138a167a
rename to sql/hive/src/test/resources/golden/udf_if-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c b/sql/hive/src/test/resources/golden/udf_if-1-b7ffa85b5785cccef2af1b285348cc2c
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c
rename to sql/hive/src/test/resources/golden/udf_if-1-b7ffa85b5785cccef2af1b285348cc2c
diff --git a/sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a b/sql/hive/src/test/resources/golden/udf_if-2-30cf7f51f92b5684e556deff3032d49a
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a
rename to sql/hive/src/test/resources/golden/udf_if-2-30cf7f51f92b5684e556deff3032d49a
diff --git a/sql/hive/src/test/resources/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 b/sql/hive/src/test/resources/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 b/sql/hive/src/test/resources/golden/udf_if-3-59e90bb74481aaf35480076806daf365
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9
rename to sql/hive/src/test/resources/golden/udf_if-3-59e90bb74481aaf35480076806daf365
diff --git a/sql/hive/src/test/resources/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca b/sql/hive/src/test/resources/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_if-3-20206f17367ff284d67044abd745ce9f b/sql/hive/src/test/resources/golden/udf_if-4-c4f4d2c83281f9c2380b5efac55fe6eb
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_if-3-20206f17367ff284d67044abd745ce9f
rename to sql/hive/src/test/resources/golden/udf_if-4-c4f4d2c83281f9c2380b5efac55fe6eb
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c b/sql/hive/src/test/resources/golden/udf_if-5-841a8068d35a42179d3654e1a2848c43
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c
rename to sql/hive/src/test/resources/golden/udf_if-5-841a8068d35a42179d3654e1a2848c43
diff --git a/sql/hive/src/test/resources/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc b/sql/hive/src/test/resources/golden/udf_if-6-508f9140dd33931c7b9ad336dceb32cf
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc
rename to sql/hive/src/test/resources/golden/udf_if-6-508f9140dd33931c7b9ad336dceb32cf
diff --git a/sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 b/sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4
deleted file mode 100644
index ee6e42ce6a83dc1c68d15505d666ddedef47a25b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4
+++ /dev/null
@@ -1 +0,0 @@
-true	false	true	true	false	true	true	true	NULL	NULL	true
diff --git a/sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 b/sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7
deleted file mode 100644
index 993d93304f95eab9a021591cdcd2a310d0c1c157..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7
+++ /dev/null
@@ -1,3 +0,0 @@
-238
-86
-238
diff --git a/sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 b/sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59
deleted file mode 100644
index cd15a08a539ab1d61e235f0e99f97d642135e67c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59
+++ /dev/null
@@ -1 +0,0 @@
-in_file(str, filename) - Returns true if str appears in the file
diff --git a/sql/hive/src/test/resources/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 b/sql/hive/src/test/resources/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 b/sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260
deleted file mode 100644
index df5a0561fb8f58011ac128529d7779f83d909ba8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260
+++ /dev/null
@@ -1 +0,0 @@
-Function '`index`' does not exist.
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_index-0-e91e3e5a22029b9b979ccbbef97add66
similarity index 100%
rename from sql/hive/src/test/resources/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/udf_index-0-e91e3e5a22029b9b979ccbbef97add66
diff --git a/sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 b/sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6
deleted file mode 100644
index df5a0561fb8f58011ac128529d7779f83d909ba8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6
+++ /dev/null
@@ -1 +0,0 @@
-Function '`index`' does not exist.
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_index-1-a277ac394cae40cb55d1ef3aa5add260 b/sql/hive/src/test/resources/golden/udf_index-1-a277ac394cae40cb55d1ef3aa5add260
new file mode 100644
index 0000000000000000000000000000000000000000..bb0a912824bbde6b346576381bd753b5b7592732
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_index-1-a277ac394cae40cb55d1ef3aa5add260
@@ -0,0 +1 @@
+Function '`index`' does not exist.
diff --git a/sql/hive/src/test/resources/golden/udf_index-2-1f5e109131b0c67ebea521fa4902a8f6 b/sql/hive/src/test/resources/golden/udf_index-2-1f5e109131b0c67ebea521fa4902a8f6
new file mode 100644
index 0000000000000000000000000000000000000000..bb0a912824bbde6b346576381bd753b5b7592732
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_index-2-1f5e109131b0c67ebea521fa4902a8f6
@@ -0,0 +1 @@
+Function '`index`' does not exist.
diff --git a/sql/hive/src/test/resources/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 b/sql/hive/src/test/resources/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922
deleted file mode 100644
index ae27b5efea045e521c98bd187dac8dc4a31caf64..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922
+++ /dev/null
@@ -1 +0,0 @@
-instr(str, substr) - Returns the index of the first occurance of substr in str
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input31-1-c21dba410fb07a098f93430a9d21df79 b/sql/hive/src/test/resources/golden/udf_instr-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/input31-1-c21dba410fb07a098f93430a9d21df79
rename to sql/hive/src/test/resources/golden/udf_instr-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922 b/sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922
new file mode 100644
index 0000000000000000000000000000000000000000..06461b525b058b42d5c4b20c61015ef858553935
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922
@@ -0,0 +1 @@
+instr(str, substr) - Returns the index of the first occurance of substr in str
diff --git a/sql/hive/src/test/resources/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 b/sql/hive/src/test/resources/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182
deleted file mode 100644
index 35de2f0d8653f4d93a13dc1a0c89fddc26d17baf..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182
+++ /dev/null
@@ -1,4 +0,0 @@
-instr(str, substr) - Returns the index of the first occurance of substr in str
-Example:
-  > SELECT instr('Facebook', 'boo') FROM src LIMIT 1;
-  5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 b/sql/hive/src/test/resources/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182 b/sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182
new file mode 100644
index 0000000000000000000000000000000000000000..5a8c34271f443a71c942c3410f5659f225bd6202
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182
@@ -0,0 +1,4 @@
+instr(str, substr) - Returns the index of the first occurance of substr in str
+Example:
+  > SELECT instr('Facebook', 'boo') FROM src LIMIT 1;
+  5
diff --git a/sql/hive/src/test/resources/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 b/sql/hive/src/test/resources/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12
deleted file mode 100644
index 4d34e6df7039b2c7262caef6cc2066f533ac9a1d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12
+++ /dev/null
@@ -1 +0,0 @@
-1	0	2	2	0	0	2	3	4	2	3	NULL	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 b/sql/hive/src/test/resources/golden/udf_instr-3-c40fbd09410b11388ce7a6e9bea5846f
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7
rename to sql/hive/src/test/resources/golden/udf_instr-3-c40fbd09410b11388ce7a6e9bea5846f
diff --git a/sql/hive/src/test/resources/golden/udf_instr-4-7017a441a31abc235d9359440cefda49 b/sql/hive/src/test/resources/golden/udf_instr-4-7017a441a31abc235d9359440cefda49
new file mode 100644
index 0000000000000000000000000000000000000000..8883b1a631ab7a2f72371e83dc0e1282c53af998
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_instr-4-7017a441a31abc235d9359440cefda49
@@ -0,0 +1 @@
+1	0	2	2	0	0	2	3	4	2	3	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba b/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba
index cda33a8e2e555a18ba297a90a0e3f1c8390ffbf3..5549ad557f6b25dbc5e70ce742a6253f51ce8074 100644
--- a/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba
+++ b/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba
@@ -1 +1 @@
-There is no documentation for function 'int'
\ No newline at end of file
+There is no documentation for function 'int'
diff --git a/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 b/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7
index cda33a8e2e555a18ba297a90a0e3f1c8390ffbf3..5549ad557f6b25dbc5e70ce742a6253f51ce8074 100644
--- a/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7
+++ b/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7
@@ -1 +1 @@
-There is no documentation for function 'int'
\ No newline at end of file
+There is no documentation for function 'int'
diff --git a/sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 b/sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6
index 723b5aaf2af6b56b9863a3cd3915d46ba01fd1e3..4ae9ea2a16e879b3b2ee0c7f88d46bb86e2a029e 100644
--- a/sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6
+++ b/sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6
@@ -1 +1 @@
-isnotnull a - Returns true if a is not NULL and false otherwise
\ No newline at end of file
+isnotnull a - Returns true if a is not NULL and false otherwise
diff --git a/sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 b/sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7
index 723b5aaf2af6b56b9863a3cd3915d46ba01fd1e3..4ae9ea2a16e879b3b2ee0c7f88d46bb86e2a029e 100644
--- a/sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7
+++ b/sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7
@@ -1 +1 @@
-isnotnull a - Returns true if a is not NULL and false otherwise
\ No newline at end of file
+isnotnull a - Returns true if a is not NULL and false otherwise
diff --git a/sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 b/sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53
index 99510c6f1db15f734807966c81ea0326fc39ce37..237cdf99168d049e34b4cfeadcd79283267c1917 100644
--- a/sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53
+++ b/sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53
@@ -1 +1 @@
-isnull a - Returns true if a is NULL and false otherwise
\ No newline at end of file
+isnull a - Returns true if a is NULL and false otherwise
diff --git a/sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 b/sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262
index 99510c6f1db15f734807966c81ea0326fc39ce37..237cdf99168d049e34b4cfeadcd79283267c1917 100644
--- a/sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262
+++ b/sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262
@@ -1 +1 @@
-isnull a - Returns true if a is NULL and false otherwise
\ No newline at end of file
+isnull a - Returns true if a is NULL and false otherwise
diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53
deleted file mode 100644
index 99510c6f1db15f734807966c81ea0326fc39ce37..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53
+++ /dev/null
@@ -1 +0,0 @@
-isnull a - Returns true if a is NULL and false otherwise
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262
deleted file mode 100644
index 99510c6f1db15f734807966c81ea0326fc39ce37..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262
+++ /dev/null
@@ -1 +0,0 @@
-isnull a - Returns true if a is NULL and false otherwise
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6
deleted file mode 100644
index 723b5aaf2af6b56b9863a3cd3915d46ba01fd1e3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6
+++ /dev/null
@@ -1 +0,0 @@
-isnotnull a - Returns true if a is not NULL and false otherwise
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7
deleted file mode 100644
index 723b5aaf2af6b56b9863a3cd3915d46ba01fd1e3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7
+++ /dev/null
@@ -1 +0,0 @@
-isnotnull a - Returns true if a is not NULL and false otherwise
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1
deleted file mode 100644
index eedfbc67d50c72d4421a98ffd069476559a733ae..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1
+++ /dev/null
@@ -1 +0,0 @@
-true	true	true
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b
deleted file mode 100644
index eedfbc67d50c72d4421a98ffd069476559a733ae..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b
+++ /dev/null
@@ -1 +0,0 @@
-true	true	true
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 b/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7
deleted file mode 100644
index b703a30abffbab14f5ca0b855d8e1d966c370ea7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7
+++ /dev/null
@@ -1 +0,0 @@
-java_method(class,method[,arg1[,arg2..]]) calls method with reflection
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a b/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a
deleted file mode 100644
index 07375dc92d625ccb6d8c582b2b4bfeb77df8f33d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a
+++ /dev/null
@@ -1,3 +0,0 @@
-java_method(class,method[,arg1[,arg2..]]) calls method with reflection
-Synonyms: reflect
-Use this UDF to call Java methods by matching the argument signature
diff --git a/sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 b/sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd b/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd
deleted file mode 100644
index 9b93703dae806ae2d392fae20fa48894eef11f8c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd
+++ /dev/null
@@ -1 +0,0 @@
-1	true	3	2	3	2.718281828459045	1.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad b/sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad
index ec6779df2818d50337c978ce11a5879b332d5f4c..afe1bf6cd351a2a71ec2e5d23b2c43e4732d01fa 100644
--- a/sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad
+++ b/sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad
@@ -1 +1 @@
-lcase(str) - Returns str with all characters changed to lowercase
\ No newline at end of file
+lcase(str) - Returns str with all characters changed to lowercase
diff --git a/sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e b/sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e
index 4da30f6c23a1b9740664d749f0be65b1f76ae090..191001deca9f7a574fa2c4c002acc7da288bc685 100644
--- a/sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e
+++ b/sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e
@@ -2,4 +2,4 @@ lcase(str) - Returns str with all characters changed to lowercase
 Synonyms: lower
 Example:
   > SELECT lcase('Facebook') FROM src LIMIT 1;
-  'facebook'
\ No newline at end of file
+  'facebook'
diff --git a/sql/hive/src/test/resources/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b b/sql/hive/src/test/resources/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b
deleted file mode 100644
index d4d2bd508b44f53ce9347028a7580043a8f72af4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b
+++ /dev/null
@@ -1 +0,0 @@
-length(str | binary) - Returns the length of str or number of bytes in binary data
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_length-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee
rename to sql/hive/src/test/resources/golden/udf_length-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_length-1-38364b60c3a2409f53c9aa2dae19903b b/sql/hive/src/test/resources/golden/udf_length-1-38364b60c3a2409f53c9aa2dae19903b
new file mode 100644
index 0000000000000000000000000000000000000000..3e2bae7430e67457c12e1d62ddee84029d171a64
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_length-1-38364b60c3a2409f53c9aa2dae19903b
@@ -0,0 +1 @@
+length(str | binary) - Returns the length of str or number of bytes in binary data
diff --git a/sql/hive/src/test/resources/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 b/sql/hive/src/test/resources/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67
deleted file mode 100644
index 79c1c54639266df2d4e1c91dd3c8d6b15f1b0cd3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67
+++ /dev/null
@@ -1,4 +0,0 @@
-length(str | binary) - Returns the length of str or number of bytes in binary data
-Example:
-  > SELECT length('Facebook') FROM src LIMIT 1;
-  8
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea b/sql/hive/src/test/resources/golden/udf_length-10-de456a5765db4a06110d9483985aa4a6
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea
rename to sql/hive/src/test/resources/golden/udf_length-10-de456a5765db4a06110d9483985aa4a6
diff --git a/sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 b/sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333
deleted file mode 100644
index d8263ee9860594d2806b0dfd1bfd17528b0ba2a4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333
+++ /dev/null
@@ -1 +0,0 @@
-2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_length-11-f3a9bd30540345db0f69b6847014b333 b/sql/hive/src/test/resources/golden/udf_length-11-f3a9bd30540345db0f69b6847014b333
new file mode 100644
index 0000000000000000000000000000000000000000..0cfbf08886fca9a91cb753ec8734c84fcbe52c9f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_length-11-f3a9bd30540345db0f69b6847014b333
@@ -0,0 +1 @@
+2
diff --git a/sql/hive/src/test/resources/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 b/sql/hive/src/test/resources/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_length-2-f183e1f8ae516bb483132ed106289b67 b/sql/hive/src/test/resources/golden/udf_length-2-f183e1f8ae516bb483132ed106289b67
new file mode 100644
index 0000000000000000000000000000000000000000..f1d3b1648718d423166a3f7ebc6946fb9ddf4197
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_length-2-f183e1f8ae516bb483132ed106289b67
@@ -0,0 +1,4 @@
+length(str | binary) - Returns the length of str or number of bytes in binary data
+Example:
+  > SELECT length('Facebook') FROM src LIMIT 1;
+  8
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 b/sql/hive/src/test/resources/golden/udf_length-3-af46cb6887618240836eaf5be8afbba6
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60
rename to sql/hive/src/test/resources/golden/udf_length-3-af46cb6887618240836eaf5be8afbba6
diff --git a/sql/hive/src/test/resources/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d b/sql/hive/src/test/resources/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 b/sql/hive/src/test/resources/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 b/sql/hive/src/test/resources/golden/udf_length-4-dcd6404afce1103d5054527e6c216d6d
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64
rename to sql/hive/src/test/resources/golden/udf_length-4-dcd6404afce1103d5054527e6c216d6d
diff --git a/sql/hive/src/test/resources/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f
deleted file mode 100644
index 6ecb2baabb29739b014056e36fe8d7cecea1c13a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f
+++ /dev/null
@@ -1,25 +0,0 @@
-7
-0
-7
-6
-7
-7
-7
-7
-6
-7
-7
-7
-7
-7
-7
-0
-0
-6
-0
-7
-7
-7
-0
-0
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 b/sql/hive/src/test/resources/golden/udf_length-5-ba49ba4e6505c74bc33b5003f3930c43
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6
rename to sql/hive/src/test/resources/golden/udf_length-5-ba49ba4e6505c74bc33b5003f3930c43
diff --git a/sql/hive/src/test/resources/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a b/sql/hive/src/test/resources/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_length-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_length-6-adc1ec67836b26b60d8547c4996bfd8f
new file mode 100644
index 0000000000000000000000000000000000000000..c77515b2f479bb0bc6a85d809dc4650be3c43e5d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_length-6-adc1ec67836b26b60d8547c4996bfd8f
@@ -0,0 +1,25 @@
+7
+0
+7
+6
+7
+7
+7
+7
+6
+7
+7
+7
+7
+7
+7
+0
+0
+6
+0
+7
+7
+7
+0
+0
+0
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 b/sql/hive/src/test/resources/golden/udf_length-7-460dffb0f1ab0ac0ebc4fd545809aa9a
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564
rename to sql/hive/src/test/resources/golden/udf_length-7-460dffb0f1ab0ac0ebc4fd545809aa9a
diff --git a/sql/hive/src/test/resources/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f b/sql/hive/src/test/resources/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf b/sql/hive/src/test/resources/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 b/sql/hive/src/test/resources/golden/udf_length-8-8f28e6c488df47d89dca670f04a7563f
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06
rename to sql/hive/src/test/resources/golden/udf_length-8-8f28e6c488df47d89dca670f04a7563f
diff --git a/sql/hive/src/test/resources/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 b/sql/hive/src/test/resources/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 b/sql/hive/src/test/resources/golden/udf_length-9-e41b220da98996f997b26ba7ef457a84
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907
rename to sql/hive/src/test/resources/golden/udf_length-9-e41b220da98996f997b26ba7ef457a84
diff --git a/sql/hive/src/test/resources/golden/input32-1-c21dba410fb07a098f93430a9d21df79 b/sql/hive/src/test/resources/golden/udf_lessthan-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/input32-1-c21dba410fb07a098f93430a9d21df79
rename to sql/hive/src/test/resources/golden/udf_lessthan-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 b/sql/hive/src/test/resources/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427
deleted file mode 100644
index b43707d550c4bf26254af4727c6d9b5913a627f5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427
+++ /dev/null
@@ -1 +0,0 @@
-a < b - Returns TRUE if a is less than b
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 b/sql/hive/src/test/resources/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142
deleted file mode 100644
index b43707d550c4bf26254af4727c6d9b5913a627f5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142
+++ /dev/null
@@ -1 +0,0 @@
-a < b - Returns TRUE if a is less than b
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-1-a0d9e8b51e3d13685b3889db38f22427 b/sql/hive/src/test/resources/golden/udf_lessthan-1-a0d9e8b51e3d13685b3889db38f22427
new file mode 100644
index 0000000000000000000000000000000000000000..d7621677e2fcefacac159389351fc045bab3abb4
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_lessthan-1-a0d9e8b51e3d13685b3889db38f22427
@@ -0,0 +1 @@
+a < b - Returns TRUE if a is less than b
diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 b/sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7
deleted file mode 100644
index 4364012093724f066d1233629dd2b272567fc5d1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7
+++ /dev/null
@@ -1 +0,0 @@
-false	true	false	false
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-2-952c655a1092a410e5346f1205cb8142 b/sql/hive/src/test/resources/golden/udf_lessthan-2-952c655a1092a410e5346f1205cb8142
new file mode 100644
index 0000000000000000000000000000000000000000..d7621677e2fcefacac159389351fc045bab3abb4
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_lessthan-2-952c655a1092a410e5346f1205cb8142
@@ -0,0 +1 @@
+a < b - Returns TRUE if a is less than b
diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-3-677a1383983c94ba8008535b5a193153 b/sql/hive/src/test/resources/golden/udf_lessthan-3-677a1383983c94ba8008535b5a193153
new file mode 100644
index 0000000000000000000000000000000000000000..5926a6ac7746d553f7d98a41562991cf516bf705
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_lessthan-3-677a1383983c94ba8008535b5a193153
@@ -0,0 +1 @@
+false	true	false	false
diff --git a/sql/hive/src/test/resources/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 b/sql/hive/src/test/resources/golden/udf_lessthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316
rename to sql/hive/src/test/resources/golden/udf_lessthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b b/sql/hive/src/test/resources/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b
deleted file mode 100644
index c36acd7ce80bc77ff5eafcec44f1f8ab63a25158..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b
+++ /dev/null
@@ -1 +0,0 @@
-a <= b - Returns TRUE if a is not greater than b
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 b/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869
deleted file mode 100644
index c36acd7ce80bc77ff5eafcec44f1f8ab63a25158..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869
+++ /dev/null
@@ -1 +0,0 @@
-a <= b - Returns TRUE if a is not greater than b
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-b3b021456c85da76d1879191886a425b b/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-b3b021456c85da76d1879191886a425b
new file mode 100644
index 0000000000000000000000000000000000000000..7f18733b6e47ed00da840738d309a9c03fc44351
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-b3b021456c85da76d1879191886a425b
@@ -0,0 +1 @@
+a <= b - Returns TRUE if a is not greater than b
diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c b/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c
deleted file mode 100644
index 38e013b6914c29c502cd964fdfa722807ea4a32f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c
+++ /dev/null
@@ -1 +0,0 @@
-false	true	true	true
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-869d983466744ad73b109211e1638869 b/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-869d983466744ad73b109211e1638869
new file mode 100644
index 0000000000000000000000000000000000000000..7f18733b6e47ed00da840738d309a9c03fc44351
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-869d983466744ad73b109211e1638869
@@ -0,0 +1 @@
+a <= b - Returns TRUE if a is not greater than b
diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-3-947dd56091ae1ef399ab32ce58317667 b/sql/hive/src/test/resources/golden/udf_lessthanorequal-3-947dd56091ae1ef399ab32ce58317667
new file mode 100644
index 0000000000000000000000000000000000000000..fc2c7b1da192033df73eb52f7f2c4b4af3d9c6da
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_lessthanorequal-3-947dd56091ae1ef399ab32ce58317667
@@ -0,0 +1 @@
+false	true	true	true
diff --git a/sql/hive/src/test/resources/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/udf_like-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/join_view-4-763ab5853bff619e6525c01e46b2a923
rename to sql/hive/src/test/resources/golden/udf_like-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 b/sql/hive/src/test/resources/golden/udf_like-1-e0ba9a953e50554bdcbc55585cffde09
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09
rename to sql/hive/src/test/resources/golden/udf_like-1-e0ba9a953e50554bdcbc55585cffde09
diff --git a/sql/hive/src/test/resources/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a b/sql/hive/src/test/resources/golden/udf_like-2-9781f89d352c506e972ad2a1d58ec03a
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a
rename to sql/hive/src/test/resources/golden/udf_like-2-9781f89d352c506e972ad2a1d58ec03a
diff --git a/sql/hive/src/test/resources/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 b/sql/hive/src/test/resources/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 b/sql/hive/src/test/resources/golden/udf_like-3-dbc46cb33f0dd356af03006d9492f8b7
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21
rename to sql/hive/src/test/resources/golden/udf_like-3-dbc46cb33f0dd356af03006d9492f8b7
diff --git a/sql/hive/src/test/resources/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc b/sql/hive/src/test/resources/golden/udf_like-4-bef03784eab9d5e8404fd24960dea4fc
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc
rename to sql/hive/src/test/resources/golden/udf_like-4-bef03784eab9d5e8404fd24960dea4fc
diff --git a/sql/hive/src/test/resources/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 b/sql/hive/src/test/resources/golden/udf_like-5-47bfd4d65090dab890b467ae06cf3bd5
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12
rename to sql/hive/src/test/resources/golden/udf_like-5-47bfd4d65090dab890b467ae06cf3bd5
diff --git a/sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 b/sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4
index 0b7372cb446d90e3e9a20659b81d40983c424d3c..134096b19d8c3cd8a4fbd7d0c58c2c70751dcfc5 100644
--- a/sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4
+++ b/sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4
@@ -1 +1 @@
-ln(x) - Returns the natural logarithm of x
\ No newline at end of file
+ln(x) - Returns the natural logarithm of x
diff --git a/sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 b/sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4
index bdf2f49ab17ca5ed7b65454884cf480e6ee87041..c4a1a46f93370daa9749df104cfd6cab1a1fe2a9 100644
--- a/sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4
+++ b/sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4
@@ -1,4 +1,4 @@
 ln(x) - Returns the natural logarithm of x
 Example:
   > SELECT ln(1) FROM src LIMIT 1;
-  0
\ No newline at end of file
+  0
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_locate-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7
rename to sql/hive/src/test/resources/golden/udf_locate-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e b/sql/hive/src/test/resources/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e
deleted file mode 100644
index 63b152162407bb7b9292ff5130bd398bff050f5f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e
+++ /dev/null
@@ -1 +0,0 @@
-locate(substr, str[, pos]) - Returns the position of the first occurance of substr in str after position pos
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e b/sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e
new file mode 100644
index 0000000000000000000000000000000000000000..84bea329540d14c27dd547c5e8cd4eb147c471b6
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e
@@ -0,0 +1 @@
+locate(substr, str[, pos]) - Returns the position of the first occurance of substr in str after position pos
diff --git a/sql/hive/src/test/resources/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 b/sql/hive/src/test/resources/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478
deleted file mode 100644
index ea5465b67683efaae1024575f8988e914ac3e128..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478
+++ /dev/null
@@ -1,4 +0,0 @@
-locate(substr, str[, pos]) - Returns the position of the first occurance of substr in str after position pos
-Example:
-  > SELECT locate('bar', 'foobarbar', 5) FROM src LIMIT 1;
-  7
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c b/sql/hive/src/test/resources/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478 b/sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478
new file mode 100644
index 0000000000000000000000000000000000000000..092e12586b9e8c394d9c35480dc3a8338c5e92db
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478
@@ -0,0 +1,4 @@
+locate(substr, str[, pos]) - Returns the position of the first occurance of substr in str after position pos
+Example:
+  > SELECT locate('bar', 'foobarbar', 5) FROM src LIMIT 1;
+  7
diff --git a/sql/hive/src/test/resources/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 b/sql/hive/src/test/resources/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531
deleted file mode 100644
index cd97bbb17fa81f6f420a4818b5f5dec6de3603b8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531
+++ /dev/null
@@ -1 +0,0 @@
-1	0	2	2	4	4	0	0	2	3	4	2	3	NULL	NULL	0	0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 b/sql/hive/src/test/resources/golden/udf_locate-3-ce4a131f99dc9befa926027380b38dbb
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981
rename to sql/hive/src/test/resources/golden/udf_locate-3-ce4a131f99dc9befa926027380b38dbb
diff --git a/sql/hive/src/test/resources/golden/udf_locate-4-104cbfb3b59ad563810ddd7304a58b1b b/sql/hive/src/test/resources/golden/udf_locate-4-104cbfb3b59ad563810ddd7304a58b1b
new file mode 100644
index 0000000000000000000000000000000000000000..f98ccc3556b422c4f5ed7894c093f60c27221ca4
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_locate-4-104cbfb3b59ad563810ddd7304a58b1b
@@ -0,0 +1 @@
+1	0	2	2	4	4	0	0	2	3	4	2	3	NULL	NULL	0	0
diff --git a/sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a b/sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a
index e0dba6b06d9a4ffb4b3e2365314dedf9f86c6448..c2c4ca2cd43852512bc0ff18d0ee1f3df52d0bff 100644
--- a/sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a
+++ b/sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a
@@ -1 +1 @@
-log([b], x) - Returns the logarithm of x with base b
\ No newline at end of file
+log([b], x) - Returns the logarithm of x with base b
diff --git a/sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 b/sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70
index 3dedfb4ddfd887e18b9e18bae26dae7fed1bba26..d6bc6ebbfd2d173362f658c5880fcc596d927126 100644
--- a/sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70
+++ b/sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70
@@ -1,4 +1,4 @@
 log([b], x) - Returns the logarithm of x with base b
 Example:
   > SELECT log(13, 13) FROM src LIMIT 1;
-  1
\ No newline at end of file
+  1
diff --git a/sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f b/sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f
index 61749c48f158edc5a308ff18bece4fdfcc31f41d..b9540eb85a3609b2a67c96b44e56d60f5dd1982a 100644
--- a/sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f
+++ b/sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f
@@ -1 +1 @@
-log10(x) - Returns the logarithm of x with base 10
\ No newline at end of file
+log10(x) - Returns the logarithm of x with base 10
diff --git a/sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 b/sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3
index 68ce39dd2b084a2726a4e9ff2f3684d608af777e..b71baf542f924f9c6981646d31f5b5440db800db 100644
--- a/sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3
+++ b/sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3
@@ -1,4 +1,4 @@
 log10(x) - Returns the logarithm of x with base 10
 Example:
   > SELECT log10(10) FROM src LIMIT 1;
-  1
\ No newline at end of file
+  1
diff --git a/sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 b/sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6
index 177008a543627177cb38a6ecedc4871cb31b04ac..8214317d87b0b7c1e0e2e129a3d2be6d630ce8f6 100644
--- a/sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6
+++ b/sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6
@@ -1 +1 @@
-log2(x) - Returns the logarithm of x with base 2
\ No newline at end of file
+log2(x) - Returns the logarithm of x with base 2
diff --git a/sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 b/sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56
index 4b856f91a7a14e4d8daab3984dea3347cdc191df..579ad959cd8844580a4c3e805822a0e3ce07d6a8 100644
--- a/sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56
+++ b/sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56
@@ -1,4 +1,4 @@
 log2(x) - Returns the logarithm of x with base 2
 Example:
   > SELECT log2(2) FROM src LIMIT 1;
-  1
\ No newline at end of file
+  1
diff --git a/sql/hive/src/test/resources/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 b/sql/hive/src/test/resources/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127
deleted file mode 100644
index d9d0dc035c9abb6915e618b45dd15a531ffa63ea..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127
+++ /dev/null
@@ -1,10 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_CREATEFUNCTION test_udf_get_java_boolean 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestGetJavaBoolean')
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-
-
diff --git a/sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 b/sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668
index 9bfc44922ea38911c0dfac27bddc4d704a14d72d..de8e1518c93d7e7903a6c81407afbb94c202342d 100644
--- a/sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668
+++ b/sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668
@@ -1 +1 @@
-lower(str) - Returns str with all characters changed to lowercase
\ No newline at end of file
+lower(str) - Returns str with all characters changed to lowercase
diff --git a/sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c b/sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c
index da912776ab009a555105f5d6540ebc5191fa32de..532f348d7da6fc76e83d01d85f7dd093c6557a99 100644
--- a/sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c
+++ b/sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c
@@ -2,4 +2,4 @@ lower(str) - Returns str with all characters changed to lowercase
 Synonyms: lcase
 Example:
   > SELECT lower('Facebook') FROM src LIMIT 1;
-  'facebook'
\ No newline at end of file
+  'facebook'
diff --git a/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 b/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08
index 9640d1794dcbc5e1436074b5c904859e12b066a4..8fbb5d4429d9670244ca13bf06980964b2b80789 100644
--- a/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08
+++ b/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08
@@ -1 +1 @@
-abc 123	ABC 123
\ No newline at end of file
+abc 123	ABC 123
diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_lpad-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe
rename to sql/hive/src/test/resources/golden/udf_lpad-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_lpad-0-995646acf1e23cea7825412915921bef b/sql/hive/src/test/resources/golden/udf_lpad-0-995646acf1e23cea7825412915921bef
deleted file mode 100644
index 672d2d07f68b505f62ce6e123626fa7e9dd7b013..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_lpad-0-995646acf1e23cea7825412915921bef
+++ /dev/null
@@ -1 +0,0 @@
-lpad(str, len, pad) - Returns str, left-padded with pad to a length of len
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_lpad-1-995646acf1e23cea7825412915921bef b/sql/hive/src/test/resources/golden/udf_lpad-1-995646acf1e23cea7825412915921bef
new file mode 100644
index 0000000000000000000000000000000000000000..e756eab5119d2758bc6e48dc552b5d620c3bbd30
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_lpad-1-995646acf1e23cea7825412915921bef
@@ -0,0 +1 @@
+lpad(str, len, pad) - Returns str, left-padded with pad to a length of len
diff --git a/sql/hive/src/test/resources/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 b/sql/hive/src/test/resources/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8
deleted file mode 100644
index 66db6624a376a47111619d516f7f390d30f21737..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8
+++ /dev/null
@@ -1,6 +0,0 @@
-lpad(str, len, pad) - Returns str, left-padded with pad to a length of len
-If str is longer than len, the return value is shortened to len characters.
-Example:
-  > SELECT lpad('hi', 5, '??') FROM src LIMIT 1;
-  '???hi'  > SELECT lpad('hi', 1, '??') FROM src LIMIT 1;
-  'h'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 b/sql/hive/src/test/resources/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_lpad-2-f58bb0fd11cb70cf197c01555ac924a8 b/sql/hive/src/test/resources/golden/udf_lpad-2-f58bb0fd11cb70cf197c01555ac924a8
new file mode 100644
index 0000000000000000000000000000000000000000..122971dbc2df93fbee0a9106fd245afde854b98a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_lpad-2-f58bb0fd11cb70cf197c01555ac924a8
@@ -0,0 +1,6 @@
+lpad(str, len, pad) - Returns str, left-padded with pad to a length of len
+If str is longer than len, the return value is shortened to len characters.
+Example:
+  > SELECT lpad('hi', 5, '??') FROM src LIMIT 1;
+  '???hi'  > SELECT lpad('hi', 1, '??') FROM src LIMIT 1;
+  'h'
diff --git a/sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae b/sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae
deleted file mode 100644
index 63a7235050df3f1ad7ce0cccc8c55962efa73c26..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae
+++ /dev/null
@@ -1 +0,0 @@
-h	...hi	1231hi
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f b/sql/hive/src/test/resources/golden/udf_lpad-3-ea9a05f035dedfe15d3a7f3d7756a2d7
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f
rename to sql/hive/src/test/resources/golden/udf_lpad-3-ea9a05f035dedfe15d3a7f3d7756a2d7
diff --git a/sql/hive/src/test/resources/golden/udf_lpad-4-48234ef55a8ec06cd4b570b9b9edab73 b/sql/hive/src/test/resources/golden/udf_lpad-4-48234ef55a8ec06cd4b570b9b9edab73
new file mode 100644
index 0000000000000000000000000000000000000000..f75d0d891636a41628ccc2972b7c3d32de627ea4
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_lpad-4-48234ef55a8ec06cd4b570b9b9edab73
@@ -0,0 +1 @@
+h	...hi	1231hi
diff --git a/sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f b/sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f
index 0c9f6fd1c269e44a4f67f88708e612face16e9e4..419d80fe502a3bb1cfa992ee025e30fc9f0fab84 100644
--- a/sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f
+++ b/sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f
@@ -1 +1 @@
-ltrim(str) - Removes the leading space characters from str 
\ No newline at end of file
+ltrim(str) - Removes the leading space characters from str 
diff --git a/sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 b/sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2
index 3ee2e690d28c18e939aa007ad71c7dc3f964ff2c..fc994e90aaa051242e88801a9b34522b66d51a28 100644
--- a/sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2
+++ b/sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2
@@ -1,4 +1,4 @@
 ltrim(str) - Removes the leading space characters from str 
 Example:
   > SELECT ltrim('   facebook') FROM src LIMIT 1;
-  'facebook'
\ No newline at end of file
+  'facebook'
diff --git a/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/udf_map-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e
rename to sql/hive/src/test/resources/golden/udf_map-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e b/sql/hive/src/test/resources/golden/udf_map-1-9feb9f29115f94b3bb4f6a36160bd17e
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e
rename to sql/hive/src/test/resources/golden/udf_map-1-9feb9f29115f94b3bb4f6a36160bd17e
diff --git a/sql/hive/src/test/resources/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf b/sql/hive/src/test/resources/golden/udf_map-2-1f8cd98df9bf7b2528506551fef87dcf
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf
rename to sql/hive/src/test/resources/golden/udf_map-2-1f8cd98df9bf7b2528506551fef87dcf
diff --git a/sql/hive/src/test/resources/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 b/sql/hive/src/test/resources/golden/udf_map-2-a3f90085abab46205e732b4c27b18340
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 b/sql/hive/src/test/resources/golden/udf_map-3-be7b52baa973b8b59b7ca63fea19aa99
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7
rename to sql/hive/src/test/resources/golden/udf_map-3-be7b52baa973b8b59b7ca63fea19aa99
diff --git a/sql/hive/src/test/resources/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 b/sql/hive/src/test/resources/golden/udf_map-4-60cb9c30285f7a9f99377ccbd143eb06
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1
rename to sql/hive/src/test/resources/golden/udf_map-4-60cb9c30285f7a9f99377ccbd143eb06
diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa b/sql/hive/src/test/resources/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa
deleted file mode 100644
index b88b74b978b8f6a0971e53fd0345c40d2762872c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa
+++ /dev/null
@@ -1 +0,0 @@
-map_keys(map) - Returns an unordered array containing the keys of the input map.
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f b/sql/hive/src/test/resources/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f
deleted file mode 100644
index b88b74b978b8f6a0971e53fd0345c40d2762872c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f
+++ /dev/null
@@ -1 +0,0 @@
-map_keys(map) - Returns an unordered array containing the keys of the input map.
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e b/sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e
deleted file mode 100644
index 3a26a2e5e94d5cf0734026357ab21a9b13d534ef..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e
+++ /dev/null
@@ -1 +0,0 @@
-[1,2,3]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a b/sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a
deleted file mode 100644
index 9b4c194f58a8eb346a69d2ddb767476c5fe52b67..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a
+++ /dev/null
@@ -1 +0,0 @@
-["b","a","c"]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 b/sql/hive/src/test/resources/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08
deleted file mode 100644
index e811d1976a24c41739fe14ee59156e5d7c1d07f9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08
+++ /dev/null
@@ -1 +0,0 @@
-map_values(map) - Returns an unordered array containing the values of the input map.
diff --git a/sql/hive/src/test/resources/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 b/sql/hive/src/test/resources/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69
deleted file mode 100644
index e811d1976a24c41739fe14ee59156e5d7c1d07f9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69
+++ /dev/null
@@ -1 +0,0 @@
-map_values(map) - Returns an unordered array containing the values of the input map.
diff --git a/sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 b/sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1
deleted file mode 100644
index 9d804e48987c6513bcdaf686f4f7b5b5987f2bb0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1
+++ /dev/null
@@ -1 +0,0 @@
-["a","b","c"]
diff --git a/sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e b/sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e
deleted file mode 100644
index be56bd1673eaac969418abc0627fee2e81fa747d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e
+++ /dev/null
@@ -1 +0,0 @@
-[2,1,3]
diff --git a/sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f b/sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f
deleted file mode 100644
index c6cd9573ed8edd2efeb2a8bb8ff2a22b04fe18a3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f
+++ /dev/null
@@ -1 +0,0 @@
-max(expr) - Returns the maximum value of expr
diff --git a/sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 b/sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9
deleted file mode 100644
index c6cd9573ed8edd2efeb2a8bb8ff2a22b04fe18a3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9
+++ /dev/null
@@ -1 +0,0 @@
-max(expr) - Returns the maximum value of expr
diff --git a/sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6
deleted file mode 100644
index 21096f0f6974986c4425b43e233af6813a9201e8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6
+++ /dev/null
@@ -1 +0,0 @@
-{"col1":498,"col2":"val_498"}	{"col1":498,"col2":"val_498"}
diff --git a/sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6
deleted file mode 100644
index 21096f0f6974986c4425b43e233af6813a9201e8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6
+++ /dev/null
@@ -1 +0,0 @@
-{"col1":498,"col2":"val_498"}	{"col1":498,"col2":"val_498"}
diff --git a/sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6
deleted file mode 100644
index 21096f0f6974986c4425b43e233af6813a9201e8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6
+++ /dev/null
@@ -1 +0,0 @@
-{"col1":498,"col2":"val_498"}	{"col1":498,"col2":"val_498"}
diff --git a/sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6
deleted file mode 100644
index 21096f0f6974986c4425b43e233af6813a9201e8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6
+++ /dev/null
@@ -1 +0,0 @@
-{"col1":498,"col2":"val_498"}	{"col1":498,"col2":"val_498"}
diff --git a/sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 b/sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8
deleted file mode 100644
index 0d59cabcde8fb230648a8fcff1690bf5e74953cb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8
+++ /dev/null
@@ -1 +0,0 @@
-min(expr) - Returns the minimum value of expr
diff --git a/sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 b/sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19
deleted file mode 100644
index 0d59cabcde8fb230648a8fcff1690bf5e74953cb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19
+++ /dev/null
@@ -1 +0,0 @@
-min(expr) - Returns the minimum value of expr
diff --git a/sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802
deleted file mode 100644
index b26a7e93c5d9f7f97b6056994de47200df62d76e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802
+++ /dev/null
@@ -1 +0,0 @@
-{"col1":0,"col2":"val_0"}	{"col1":0,"col2":"val_0"}
diff --git a/sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802
deleted file mode 100644
index b26a7e93c5d9f7f97b6056994de47200df62d76e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802
+++ /dev/null
@@ -1 +0,0 @@
-{"col1":0,"col2":"val_0"}	{"col1":0,"col2":"val_0"}
diff --git a/sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802
deleted file mode 100644
index b26a7e93c5d9f7f97b6056994de47200df62d76e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802
+++ /dev/null
@@ -1 +0,0 @@
-{"col1":0,"col2":"val_0"}	{"col1":0,"col2":"val_0"}
diff --git a/sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802
deleted file mode 100644
index b26a7e93c5d9f7f97b6056994de47200df62d76e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802
+++ /dev/null
@@ -1 +0,0 @@
-{"col1":0,"col2":"val_0"}	{"col1":0,"col2":"val_0"}
diff --git a/sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee b/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee
index b6bd6049a90b922b90a6a3bc6434ea8f1e69034a..231e4f382566dc0154898fb5c4c0f2cdad073c6b 100644
--- a/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee
+++ b/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee
@@ -1 +1 @@
-minute(date) - Returns the minute of date
\ No newline at end of file
+minute(date) - Returns the minute of date
diff --git a/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 b/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48
index 64c3cda697e1e2a0e376d0bf22d4472c24e02145..ea842ea174ae48809da65678d9e86fccc823ed81 100644
--- a/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48
+++ b/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48
@@ -4,4 +4,4 @@ Example:
    > SELECT minute('2009-07-30 12:58:59') FROM src LIMIT 1;
   58
   > SELECT minute('12:58:59') FROM src LIMIT 1;
-  58
\ No newline at end of file
+  58
diff --git a/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c b/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c
index 5bd5433375d5c31b341ac629a6214aa1804ad03b..e9ca4578409c882e492594732b1aa0fd85109a56 100644
--- a/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c
+++ b/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c
@@ -1 +1 @@
-14	14	NULL
\ No newline at end of file
+14	14	NULL
diff --git a/sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 b/sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5
index 1bcf0c71008b0977dd4b68acd3500f5b2def9507..6df0bfb4fecf87f2a1561dce7757c971379257d1 100644
--- a/sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5
+++ b/sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5
@@ -1 +1 @@
-a % b - Returns the remainder when dividing a by b
\ No newline at end of file
+a % b - Returns the remainder when dividing a by b
diff --git a/sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 b/sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877
index 1bcf0c71008b0977dd4b68acd3500f5b2def9507..6df0bfb4fecf87f2a1561dce7757c971379257d1 100644
--- a/sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877
+++ b/sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877
@@ -1 +1 @@
-a % b - Returns the remainder when dividing a by b
\ No newline at end of file
+a % b - Returns the remainder when dividing a by b
diff --git a/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee b/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee
index b6bd6049a90b922b90a6a3bc6434ea8f1e69034a..231e4f382566dc0154898fb5c4c0f2cdad073c6b 100644
--- a/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee
+++ b/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee
@@ -1 +1 @@
-minute(date) - Returns the minute of date
\ No newline at end of file
+minute(date) - Returns the minute of date
diff --git a/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 b/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48
index 64c3cda697e1e2a0e376d0bf22d4472c24e02145..ea842ea174ae48809da65678d9e86fccc823ed81 100644
--- a/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48
+++ b/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48
@@ -4,4 +4,4 @@ Example:
    > SELECT minute('2009-07-30 12:58:59') FROM src LIMIT 1;
   58
   > SELECT minute('12:58:59') FROM src LIMIT 1;
-  58
\ No newline at end of file
+  58
diff --git a/sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 b/sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8
deleted file mode 100644
index 3142f7f94be9e520d528c43e8ca2dc3cfee967d1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8
+++ /dev/null
@@ -1 +0,0 @@
-negative a - Returns -a
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/udf_negative-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e
rename to sql/hive/src/test/resources/golden/udf_negative-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_negative-1-1b770ec6fb07bb771af2231a9723ec8 b/sql/hive/src/test/resources/golden/udf_negative-1-1b770ec6fb07bb771af2231a9723ec8
new file mode 100644
index 0000000000000000000000000000000000000000..7ba1b77782f07677199dace850fdefaec78312ea
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_negative-1-1b770ec6fb07bb771af2231a9723ec8
@@ -0,0 +1 @@
+negative a - Returns -a
diff --git a/sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf b/sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf
deleted file mode 100644
index 3142f7f94be9e520d528c43e8ca2dc3cfee967d1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf
+++ /dev/null
@@ -1 +0,0 @@
-negative a - Returns -a
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8 b/sql/hive/src/test/resources/golden/udf_negative-10-1cd28efecc0d26f463221195f5e39956
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8
rename to sql/hive/src/test/resources/golden/udf_negative-10-1cd28efecc0d26f463221195f5e39956
diff --git a/sql/hive/src/test/resources/golden/udf_negative-2-5f64266721b1ed31cfe84ee2f2377bdf b/sql/hive/src/test/resources/golden/udf_negative-2-5f64266721b1ed31cfe84ee2f2377bdf
new file mode 100644
index 0000000000000000000000000000000000000000..7ba1b77782f07677199dace850fdefaec78312ea
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_negative-2-5f64266721b1ed31cfe84ee2f2377bdf
@@ -0,0 +1 @@
+negative a - Returns -a
diff --git a/sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c b/sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c
deleted file mode 100644
index 0f4a4dce9dd3d3809a9d5ca451b7e3fbc59b3ab1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c
+++ /dev/null
@@ -1 +0,0 @@
-a - b - Returns the difference a-b
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_negative-3-a6863d2c5fc8c3131fe70080a011392c b/sql/hive/src/test/resources/golden/udf_negative-3-a6863d2c5fc8c3131fe70080a011392c
new file mode 100644
index 0000000000000000000000000000000000000000..1ad8be1242f0010f069fddeaf91527be54d49f3e
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_negative-3-a6863d2c5fc8c3131fe70080a011392c
@@ -0,0 +1 @@
+a - b - Returns the difference a-b
diff --git a/sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 b/sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722
deleted file mode 100644
index 0f4a4dce9dd3d3809a9d5ca451b7e3fbc59b3ab1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722
+++ /dev/null
@@ -1 +0,0 @@
-a - b - Returns the difference a-b
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_negative-4-b90eec030fee9cbd177f9615b782d722 b/sql/hive/src/test/resources/golden/udf_negative-4-b90eec030fee9cbd177f9615b782d722
new file mode 100644
index 0000000000000000000000000000000000000000..1ad8be1242f0010f069fddeaf91527be54d49f3e
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_negative-4-b90eec030fee9cbd177f9615b782d722
@@ -0,0 +1 @@
+a - b - Returns the difference a-b
diff --git a/sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c b/sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a b/sql/hive/src/test/resources/golden/udf_negative-5-771e76b0acd8ddb128781da7819d0e47
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a
rename to sql/hive/src/test/resources/golden/udf_negative-5-771e76b0acd8ddb128781da7819d0e47
diff --git a/sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e b/sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 b/sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d b/sql/hive/src/test/resources/golden/udf_negative-6-f62c4a097c592871d896a7dc47c42f61
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d
rename to sql/hive/src/test/resources/golden/udf_negative-6-f62c4a097c592871d896a7dc47c42f61
diff --git a/sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b b/sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-23-828558020ce907ffa7e847762a5e2358 b/sql/hive/src/test/resources/golden/udf_negative-7-f838053f5ca5c8746dc299473dff0490
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-23-828558020ce907ffa7e847762a5e2358
rename to sql/hive/src/test/resources/golden/udf_negative-7-f838053f5ca5c8746dc299473dff0490
diff --git a/sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 b/sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da b/sql/hive/src/test/resources/golden/udf_negative-8-f4f23aa6f634913d194a69261af8f3f6
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da
rename to sql/hive/src/test/resources/golden/udf_negative-8-f4f23aa6f634913d194a69261af8f3f6
diff --git a/sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa b/sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635 b/sql/hive/src/test/resources/golden/udf_negative-9-f6a78fa3ea0f519d0e4abc5be7a960e5
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635
rename to sql/hive/src/test/resources/golden/udf_negative-9-f6a78fa3ea0f519d0e4abc5be7a960e5
diff --git a/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc b/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc
index 1ad92784d47ac9db64a11236cf86dda5c813681f..85b491240e71123a3e49cc80313c0a088f916427 100644
--- a/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc
+++ b/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc
@@ -1 +1 @@
-not a - Logical not
\ No newline at end of file
+not a - Logical not
diff --git a/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 b/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911
index 6520d74f02566bcdde937c0ad3555bd8988cf862..0956574686405c1cb79826285f8205c9ca448c10 100644
--- a/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911
+++ b/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911
@@ -1,2 +1,2 @@
 not a - Logical not
-Synonyms: !
\ No newline at end of file
+Synonyms: !
diff --git a/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 b/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117
index c0e84e95d01c2f00aa7710b412ac8ad2d5033acc..e6b0fa3967745ecfa3ef5b529a8097b2ecc008c4 100644
--- a/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117
+++ b/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117
@@ -1 +1 @@
-! a - Logical not
\ No newline at end of file
+! a - Logical not
diff --git a/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a b/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a
index 05cb1f6ab03d79977a866d828bac3e403eed3dd0..00f749069b4a3628860074040aad794170abdd0f 100644
--- a/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a
+++ b/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a
@@ -1,2 +1,2 @@
 ! a - Logical not
-Synonyms: not
\ No newline at end of file
+Synonyms: not
diff --git a/sql/hive/src/test/resources/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e b/sql/hive/src/test/resources/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e
deleted file mode 100644
index 9e2108ff72b18a10d569e6e19e3574b7d5636156..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e
+++ /dev/null
@@ -1 +0,0 @@
-a <> b - Returns TRUE if a is not equal to b
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/udf_notequal-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e
rename to sql/hive/src/test/resources/golden/udf_notequal-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_notequal-1-27c0d57f7c7c48ceb7bc671f7160254e b/sql/hive/src/test/resources/golden/udf_notequal-1-27c0d57f7c7c48ceb7bc671f7160254e
new file mode 100644
index 0000000000000000000000000000000000000000..5bcfd313a8ff67bac59e1aca08416b36f125ad09
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_notequal-1-27c0d57f7c7c48ceb7bc671f7160254e
@@ -0,0 +1 @@
+a <> b - Returns TRUE if a is not equal to b
diff --git a/sql/hive/src/test/resources/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a b/sql/hive/src/test/resources/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a
deleted file mode 100644
index 162f808ed5b67a0d38a3f06ce4578ddf543989c3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a
+++ /dev/null
@@ -1,2 +0,0 @@
-a <> b - Returns TRUE if a is not equal to b
-Synonyms: !=
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 b/sql/hive/src/test/resources/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520
deleted file mode 100644
index 2dab01a3b2765e2a0297778bfa7a5aff01c521f9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520
+++ /dev/null
@@ -1 +0,0 @@
-a != b - Returns TRUE if a is not equal to b
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_notequal-2-a7f0d1064f8f398ef504008015fddf9a b/sql/hive/src/test/resources/golden/udf_notequal-2-a7f0d1064f8f398ef504008015fddf9a
new file mode 100644
index 0000000000000000000000000000000000000000..d8be357a858c4c11e4abd85ded477e73072722d5
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_notequal-2-a7f0d1064f8f398ef504008015fddf9a
@@ -0,0 +1,2 @@
+a <> b - Returns TRUE if a is not equal to b
+Synonyms: !=
diff --git a/sql/hive/src/test/resources/golden/udf_notequal-3-7d815b1218c85e4cf69d6780cab17520 b/sql/hive/src/test/resources/golden/udf_notequal-3-7d815b1218c85e4cf69d6780cab17520
new file mode 100644
index 0000000000000000000000000000000000000000..aeffc394679cf57e3d4e006944668fba04ad4156
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_notequal-3-7d815b1218c85e4cf69d6780cab17520
@@ -0,0 +1 @@
+a != b - Returns TRUE if a is not equal to b
diff --git a/sql/hive/src/test/resources/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 b/sql/hive/src/test/resources/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57
deleted file mode 100644
index 1f73486ba28963683d1166992d844c783483c668..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57
+++ /dev/null
@@ -1,2 +0,0 @@
-a != b - Returns TRUE if a is not equal to b
-Synonyms: <>
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_notequal-4-b72baeb22fad61bb31ce2d2e69375f57 b/sql/hive/src/test/resources/golden/udf_notequal-4-b72baeb22fad61bb31ce2d2e69375f57
new file mode 100644
index 0000000000000000000000000000000000000000..32e73741d4cbc451ef65202d294e234ac5a1624a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_notequal-4-b72baeb22fad61bb31ce2d2e69375f57
@@ -0,0 +1,2 @@
+a != b - Returns TRUE if a is not equal to b
+Synonyms: <>
diff --git a/sql/hive/src/test/resources/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 b/sql/hive/src/test/resources/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 b/sql/hive/src/test/resources/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07
deleted file mode 100644
index f82286d8bb0697c9074f25b0cc189e657b6f8010..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07
+++ /dev/null
@@ -1,499 +0,0 @@
-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
-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
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 b/sql/hive/src/test/resources/golden/udf_notequal-5-eb04e5ee00967799c913e8a5b424a332
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387
rename to sql/hive/src/test/resources/golden/udf_notequal-5-eb04e5ee00967799c913e8a5b424a332
diff --git a/sql/hive/src/test/resources/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 b/sql/hive/src/test/resources/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_notequal-6-e361b9cf294c4aa25018b081a2c05e07 b/sql/hive/src/test/resources/golden/udf_notequal-6-e361b9cf294c4aa25018b081a2c05e07
new file mode 100644
index 0000000000000000000000000000000000000000..36244b327dc2f3b97dc79d67c79d3f5c504e5b72
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_notequal-6-e361b9cf294c4aa25018b081a2c05e07
@@ -0,0 +1,499 @@
+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
+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/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f b/sql/hive/src/test/resources/golden/udf_notequal-7-46a6514f2d7e6a097035ec1559df0096
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f
rename to sql/hive/src/test/resources/golden/udf_notequal-7-46a6514f2d7e6a097035ec1559df0096
diff --git a/sql/hive/src/test/resources/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd b/sql/hive/src/test/resources/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd
deleted file mode 100644
index f82286d8bb0697c9074f25b0cc189e657b6f8010..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd
+++ /dev/null
@@ -1,499 +0,0 @@
-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
-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
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_notequal-8-a71fea4e9514cda0da9542a7701613dd b/sql/hive/src/test/resources/golden/udf_notequal-8-a71fea4e9514cda0da9542a7701613dd
new file mode 100644
index 0000000000000000000000000000000000000000..36244b327dc2f3b97dc79d67c79d3f5c504e5b72
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_notequal-8-a71fea4e9514cda0da9542a7701613dd
@@ -0,0 +1,499 @@
+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
+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/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 b/sql/hive/src/test/resources/golden/udf_notop-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37
rename to sql/hive/src/test/resources/golden/udf_notop-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 b/sql/hive/src/test/resources/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6
deleted file mode 100644
index a55e3339049e819e284c83420dfdbeb0ca4b6294..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6
+++ /dev/null
@@ -1 +0,0 @@
-false	true	false	false	true	false	true	false	true	true
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_notop-1-1ce21a9b4492969c1a97612b0ccc19f2 b/sql/hive/src/test/resources/golden/udf_notop-1-1ce21a9b4492969c1a97612b0ccc19f2
new file mode 100644
index 0000000000000000000000000000000000000000..5ffd61b380318809f6e54bcd83cee6cfa730b9d4
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_notop-1-1ce21a9b4492969c1a97612b0ccc19f2
@@ -0,0 +1 @@
+false	true	false	false	true	false	true	false	true	true
diff --git a/sql/hive/src/test/resources/golden/lock3-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_nvl-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/lock3-3-16367c381d4b189b3640c92511244bfe
rename to sql/hive/src/test/resources/golden/udf_nvl-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 b/sql/hive/src/test/resources/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4
deleted file mode 100644
index 5ffcb851d56f93bcaa209d270efc19e3824a0102..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4
+++ /dev/null
@@ -1 +0,0 @@
-nvl(value,default_value) - Returns default value if value is null else returns value
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 b/sql/hive/src/test/resources/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092
deleted file mode 100644
index 5afff220da695a53bca83cd4e107391fd2b18ca8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092
+++ /dev/null
@@ -1,4 +0,0 @@
-nvl(value,default_value) - Returns default value if value is null else returns value
-Example:
-  > SELECT nvl(null,'bla') FROM src LIMIT 1;
-  bla
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_nvl-1-df7383141df0bb71ccb57f3eef9775b4 b/sql/hive/src/test/resources/golden/udf_nvl-1-df7383141df0bb71ccb57f3eef9775b4
new file mode 100644
index 0000000000000000000000000000000000000000..b043150b9c901f92d8714b8652e08ef9a5fd26f1
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_nvl-1-df7383141df0bb71ccb57f3eef9775b4
@@ -0,0 +1 @@
+nvl(value,default_value) - Returns default value if value is null else returns value
diff --git a/sql/hive/src/test/resources/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def b/sql/hive/src/test/resources/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_nvl-2-6ec6c4b23c742fc604c9937a25b0b092 b/sql/hive/src/test/resources/golden/udf_nvl-2-6ec6c4b23c742fc604c9937a25b0b092
new file mode 100644
index 0000000000000000000000000000000000000000..4daa1bbffa621d0db0ab9986c644105a9f918145
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_nvl-2-6ec6c4b23c742fc604c9937a25b0b092
@@ -0,0 +1,4 @@
+nvl(value,default_value) - Returns default value if value is null else returns value
+Example:
+  > SELECT nvl(null,'bla') FROM src LIMIT 1;
+  bla
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef b/sql/hive/src/test/resources/golden/udf_nvl-3-47199a1c23cb1cc6827c601bb66513d3
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef
rename to sql/hive/src/test/resources/golden/udf_nvl-3-47199a1c23cb1cc6827c601bb66513d3
diff --git a/sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 b/sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0
deleted file mode 100644
index 2087e1749445900c80a5c200f2ad04032205f278..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0
+++ /dev/null
@@ -1 +0,0 @@
-1	5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_nvl-4-656661e80deb75729fef313d5e2bd330 b/sql/hive/src/test/resources/golden/udf_nvl-4-656661e80deb75729fef313d5e2bd330
new file mode 100644
index 0000000000000000000000000000000000000000..273bc7331072adc0ce98904f4d87603c4380b6c9
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_nvl-4-656661e80deb75729fef313d5e2bd330
@@ -0,0 +1 @@
+1	5
diff --git a/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 b/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70
index 9138f44ad2a411207b8f7478d4f7d70af7804d54..f0e20ea40509ce1489f5db71fcc7f1876b88eb72 100644
--- a/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70
+++ b/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70
@@ -1 +1 @@
-a or b - Logical or
\ No newline at end of file
+a or b - Logical or
diff --git a/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 b/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0
index 9138f44ad2a411207b8f7478d4f7d70af7804d54..f0e20ea40509ce1489f5db71fcc7f1876b88eb72 100644
--- a/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0
+++ b/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0
@@ -1 +1 @@
-a or b - Logical or
\ No newline at end of file
+a or b - Logical or
diff --git a/sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 b/sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8
index 0535b085e50c2018105f4b8e36a6b2357d5f1bb7..1ce92fd7158222aef9bcebea574989bbad3ee599 100644
--- a/sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8
+++ b/sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8
@@ -1 +1 @@
-parse_url(url, partToExtract[, key]) - extracts a part from a URL
\ No newline at end of file
+parse_url(url, partToExtract[, key]) - extracts a part from a URL
diff --git a/sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab b/sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab
index 7178e07a07e48f8cc447c2e01f89db4c092cec36..bd448a4e4cade0c1ff7426fda514af5a07dd5971 100644
--- a/sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab
+++ b/sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab
@@ -7,4 +7,4 @@ Example:
   > SELECT parse_url('http://facebook.com/path/p1.php?query=1', 'QUERY') FROM src LIMIT 1;
   'query=1'
   > SELECT parse_url('http://facebook.com/path/p1.php?query=1', 'QUERY', 'query') FROM src LIMIT 1;
-  '1'
\ No newline at end of file
+  '1'
diff --git a/sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 b/sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2
index f5483d4e3dafe2832fe7f703cd79bf66a66a408b..e3ece483b53fdb5ba95154aaa51db5be0ecbed96 100644
--- a/sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2
+++ b/sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2
@@ -1 +1 @@
-facebook.com	/path1/p.php	k1=v1&k2=v2	Ref1	v2	v1	NULL	/path1/p.php?k1=v1&k2=v2	http	NULL	facebook.com
\ No newline at end of file
+facebook.com	/path1/p.php	k1=v1&k2=v2	Ref1	v2	v1	NULL	/path1/p.php?k1=v1&k2=v2	http	NULL	facebook.com
diff --git a/sql/hive/src/test/resources/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 b/sql/hive/src/test/resources/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553
deleted file mode 100644
index 2025042f5d493abe0de3899abf8abcbe88540c7d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553
+++ /dev/null
@@ -1 +0,0 @@
-percentile(expr, pc) - Returns the percentile(s) of expr at pc (range: [0,1]).pc can be a double or double array
diff --git a/sql/hive/src/test/resources/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a b/sql/hive/src/test/resources/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a
deleted file mode 100644
index 2025042f5d493abe0de3899abf8abcbe88540c7d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a
+++ /dev/null
@@ -1 +0,0 @@
-percentile(expr, pc) - Returns the percentile(s) of expr at pc (range: [0,1]).pc can be a double or double array
diff --git a/sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 b/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1
deleted file mode 100644
index 5d2fc352ee060e0b30f4b3e2b11d298ac55684f8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1
+++ /dev/null
@@ -1 +0,0 @@
-a pmod b - Compute the positive modulo
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae b/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae
deleted file mode 100644
index 5d2fc352ee060e0b30f4b3e2b11d298ac55684f8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae
+++ /dev/null
@@ -1 +0,0 @@
-a pmod b - Compute the positive modulo
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 b/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2
deleted file mode 100644
index 0b46af11c4516bee251f699251434a53cd3f8784..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2
+++ /dev/null
@@ -1 +0,0 @@
-6.89	51.7	18.09
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 b/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234
deleted file mode 100644
index 5eb0813b60eb6737f7c0e8200eeeb2408f75c6e5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234
+++ /dev/null
@@ -1 +0,0 @@
-8	51	15
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 b/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1
deleted file mode 100644
index e21e4b08e7a62ae229be34bb9f9371c287fad557..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1
+++ /dev/null
@@ -1 +0,0 @@
-5	50	0
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 b/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029
deleted file mode 100644
index e0bc2a844fb46e43dcd3ffffa35c42ab73226a83..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029
+++ /dev/null
@@ -1 +0,0 @@
-8	51	16
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 b/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125
deleted file mode 100644
index e0bc2a844fb46e43dcd3ffffa35c42ab73226a83..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125
+++ /dev/null
@@ -1 +0,0 @@
-8	51	16
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 b/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618
deleted file mode 100644
index e0bc2a844fb46e43dcd3ffffa35c42ab73226a83..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618
+++ /dev/null
@@ -1 +0,0 @@
-8	51	16
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd b/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd
deleted file mode 100644
index 48371142e9b5d984ca632ff8bc7d01a643d6855d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd
+++ /dev/null
@@ -1 +0,0 @@
-6.8899984	51.700005	18.089996
diff --git a/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 b/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57
deleted file mode 100644
index ab842acd48b3c4f9ba17ca8d5ff47c346ac1d326..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57
+++ /dev/null
@@ -1 +0,0 @@
-6.890000000000011	51.699999999999996	18.090000000000003
diff --git a/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 b/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2
index 7c98729cc195c4a8776f793776ca1d7503d7d501..6374b08a607abe004972cac448873210cc762073 100644
--- a/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2
+++ b/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2
@@ -1 +1 @@
-positive a - Returns a
\ No newline at end of file
+positive a - Returns a
diff --git a/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c b/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c
index 7c98729cc195c4a8776f793776ca1d7503d7d501..6374b08a607abe004972cac448873210cc762073 100644
--- a/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c
+++ b/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c
@@ -1 +1 @@
-positive a - Returns a
\ No newline at end of file
+positive a - Returns a
diff --git a/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 b/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23
index e8f11c444a808d7aae34306e037017d597d783d5..9d794539b4ae0a5393e482ec39ca108997643290 100644
--- a/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23
+++ b/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23
@@ -1 +1 @@
-a + b - Returns a+b
\ No newline at end of file
+a + b - Returns a+b
diff --git a/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 b/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5
index e8f11c444a808d7aae34306e037017d597d783d5..9d794539b4ae0a5393e482ec39ca108997643290 100644
--- a/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5
+++ b/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5
@@ -1 +1 @@
-a + b - Returns a+b
\ No newline at end of file
+a + b - Returns a+b
diff --git a/sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 b/sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81
index 43f197e360c827ec5d3d7bf73185a22790c34b79..67377963d2aa132a4a1f192d40710ff83f3acb21 100644
--- a/sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81
+++ b/sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81
@@ -1 +1 @@
-pow(x1, x2) - raise x1 to the power of x2
\ No newline at end of file
+pow(x1, x2) - raise x1 to the power of x2
diff --git a/sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 b/sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803
index afc2e4462a1a8f09a273b4970b06e66bafb9d170..ded9d10a595ff1afe458c7957789fdc1f457229b 100644
--- a/sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803
+++ b/sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803
@@ -2,4 +2,4 @@ pow(x1, x2) - raise x1 to the power of x2
 Synonyms: power
 Example:
   > SELECT pow(2, 3) FROM src LIMIT 1;
-  8
\ No newline at end of file
+  8
diff --git a/sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4 b/sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4
index 5e3a6a8f31fd3af926d357de4f6d0f0a42d898dd..90e23c3255b77310670af176b9472388098f38af 100644
--- a/sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4
+++ b/sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4
@@ -1 +1 @@
-power(x1, x2) - raise x1 to the power of x2
\ No newline at end of file
+power(x1, x2) - raise x1 to the power of x2
diff --git a/sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 b/sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426
index c3414e29768e342a0023d64f741ba6cdfd570c78..4890e2e989d349dacfa5b6e47c56470c1c730645 100644
--- a/sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426
+++ b/sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426
@@ -2,4 +2,4 @@ power(x1, x2) - raise x1 to the power of x2
 Synonyms: pow
 Example:
   > SELECT power(2, 3) FROM src LIMIT 1;
-  8
\ No newline at end of file
+  8
diff --git a/sql/hive/src/test/resources/golden/udf_printf-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_printf-0-e86d559aeb84a4cc017a103182c22bfb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e b/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e
deleted file mode 100644
index 1635ff88dd76818df370280f6de316ac85a1f949..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e
+++ /dev/null
@@ -1 +0,0 @@
-printf(String format, Obj... args) - function that can format strings according to printf-style format strings
diff --git a/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee b/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee
deleted file mode 100644
index 62440ee68e145644f40124a9493cbbace35fce6b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee
+++ /dev/null
@@ -1,4 +0,0 @@
-printf(String format, Obj... args) - function that can format strings according to printf-style format strings
-Example:
-  > SELECT printf("Hello World %d %s", 100, "days")FROM src LIMIT 1;
-  "Hello World 100 days"
diff --git a/sql/hive/src/test/resources/golden/udf_printf-3-9c568a0473888396bd46507e8b330c36 b/sql/hive/src/test/resources/golden/udf_printf-3-9c568a0473888396bd46507e8b330c36
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475 b/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475
deleted file mode 100644
index 39cb945991403000fc5f3a64d5d41b634ca6a25b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475
+++ /dev/null
@@ -1 +0,0 @@
-Hello World 100 days
diff --git a/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb b/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb
deleted file mode 100644
index 04bf5e552a576c2988637ecc3714a4d84dd3858b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb
+++ /dev/null
@@ -1 +0,0 @@
-All Type Test: false, A, 15000, 1.234000e+01, +27183.2401, 2300.41, 32, corret, 0x1.002p8
diff --git a/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294 b/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294
deleted file mode 100644
index 2e9f7509968a397166a55f8b6398e264f0d8c219..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294
+++ /dev/null
@@ -1 +0,0 @@
-Color red, String Null: null, number1 123456, number2 00089, Integer Null: null, hex 0xff, float  3.14 Double Null: null
diff --git a/sql/hive/src/test/resources/golden/udf_printf-7-5769f3a5b3300ca1d8b861229e976126 b/sql/hive/src/test/resources/golden/udf_printf-7-5769f3a5b3300ca1d8b861229e976126
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_radians-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7
rename to sql/hive/src/test/resources/golden/udf_radians-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 b/sql/hive/src/test/resources/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 b/sql/hive/src/test/resources/golden/udf_radians-1-58b73fc96927d447d1225f021eaa378
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968
rename to sql/hive/src/test/resources/golden/udf_radians-1-58b73fc96927d447d1225f021eaa378
diff --git a/sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 b/sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93
deleted file mode 100644
index 116b6bc461ed09ab95eabcaed5c9418ce6c03349..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93
+++ /dev/null
@@ -1 +0,0 @@
-1.000000357564167
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_radians-10-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/sql/hive/src/test/resources/golden/udf_radians-10-9f1b8ddf9be2e5f9b9b7ff4f00cfb66
new file mode 100644
index 0000000000000000000000000000000000000000..b00bf83e61d8256a5d225c1aa3b983679f840f58
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_radians-10-9f1b8ddf9be2e5f9b9b7ff4f00cfb66
@@ -0,0 +1,5 @@
+radians(x) - Converts degrees to radians
+Example:
+  > SELECT radians(90) FROM src LIMIT 1;
+  1.5707963267949mo
+
diff --git a/sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 b/sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3
deleted file mode 100644
index aaf3b31fef48889a81099cc13431c54d21c24144..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3
+++ /dev/null
@@ -1 +0,0 @@
-2.4999991485811655
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_radians-2-cb8462f25c18b7405c41a50e52476d04 b/sql/hive/src/test/resources/golden/udf_radians-2-cb8462f25c18b7405c41a50e52476d04
new file mode 100644
index 0000000000000000000000000000000000000000..6b0996864478b8f1f6e5448439212133ecc3696c
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_radians-2-cb8462f25c18b7405c41a50e52476d04
@@ -0,0 +1 @@
+1.000000357564167
diff --git a/sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 b/sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14
deleted file mode 100644
index 73ad88be4ef3d74d76458055afae541a071be751..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14
+++ /dev/null
@@ -1 +0,0 @@
-radians(x) - Converts degrees to radians
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_radians-3-bd00297cb26f599913b14a635e768be3 b/sql/hive/src/test/resources/golden/udf_radians-3-bd00297cb26f599913b14a635e768be3
new file mode 100644
index 0000000000000000000000000000000000000000..1f204866982ad6e11bbf1bb8b2264d2bbceea44a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_radians-3-bd00297cb26f599913b14a635e768be3
@@ -0,0 +1 @@
+2.4999991485811655
diff --git a/sql/hive/src/test/resources/golden/udf_radians-4-65e16c7b13de48a5d36793d0c7d35e14 b/sql/hive/src/test/resources/golden/udf_radians-4-65e16c7b13de48a5d36793d0c7d35e14
new file mode 100644
index 0000000000000000000000000000000000000000..99e71b16f18ad88b2cad8517f492eb128bc7a130
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_radians-4-65e16c7b13de48a5d36793d0c7d35e14
@@ -0,0 +1 @@
+radians(x) - Converts degrees to radians
diff --git a/sql/hive/src/test/resources/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/sql/hive/src/test/resources/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66
deleted file mode 100644
index e0237c0058f55a841f121be33d902fb3817fcb79..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66
+++ /dev/null
@@ -1,4 +0,0 @@
-radians(x) - Converts degrees to radians
-Example:
-  > SELECT radians(90) FROM src LIMIT 1;
-  1.5707963267949mo
diff --git a/sql/hive/src/test/resources/golden/udf_radians-5-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/sql/hive/src/test/resources/golden/udf_radians-5-9f1b8ddf9be2e5f9b9b7ff4f00cfb66
new file mode 100644
index 0000000000000000000000000000000000000000..b00bf83e61d8256a5d225c1aa3b983679f840f58
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_radians-5-9f1b8ddf9be2e5f9b9b7ff4f00cfb66
@@ -0,0 +1,5 @@
+radians(x) - Converts degrees to radians
+Example:
+  > SELECT radians(90) FROM src LIMIT 1;
+  1.5707963267949mo
+
diff --git a/sql/hive/src/test/resources/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 b/sql/hive/src/test/resources/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 b/sql/hive/src/test/resources/golden/udf_radians-6-70c9e7199b5898e2c3a4943ec58da113
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610
rename to sql/hive/src/test/resources/golden/udf_radians-6-70c9e7199b5898e2c3a4943ec58da113
diff --git a/sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 b/sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93
deleted file mode 100644
index 116b6bc461ed09ab95eabcaed5c9418ce6c03349..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93
+++ /dev/null
@@ -1 +0,0 @@
-1.000000357564167
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 b/sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3
deleted file mode 100644
index aaf3b31fef48889a81099cc13431c54d21c24144..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3
+++ /dev/null
@@ -1 +0,0 @@
-2.4999991485811655
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_radians-7-cb8462f25c18b7405c41a50e52476d04 b/sql/hive/src/test/resources/golden/udf_radians-7-cb8462f25c18b7405c41a50e52476d04
new file mode 100644
index 0000000000000000000000000000000000000000..6b0996864478b8f1f6e5448439212133ecc3696c
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_radians-7-cb8462f25c18b7405c41a50e52476d04
@@ -0,0 +1 @@
+1.000000357564167
diff --git a/sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 b/sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14
deleted file mode 100644
index 73ad88be4ef3d74d76458055afae541a071be751..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14
+++ /dev/null
@@ -1 +0,0 @@
-radians(x) - Converts degrees to radians
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_radians-8-bd00297cb26f599913b14a635e768be3 b/sql/hive/src/test/resources/golden/udf_radians-8-bd00297cb26f599913b14a635e768be3
new file mode 100644
index 0000000000000000000000000000000000000000..1f204866982ad6e11bbf1bb8b2264d2bbceea44a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_radians-8-bd00297cb26f599913b14a635e768be3
@@ -0,0 +1 @@
+2.4999991485811655
diff --git a/sql/hive/src/test/resources/golden/udf_radians-9-65e16c7b13de48a5d36793d0c7d35e14 b/sql/hive/src/test/resources/golden/udf_radians-9-65e16c7b13de48a5d36793d0c7d35e14
new file mode 100644
index 0000000000000000000000000000000000000000..99e71b16f18ad88b2cad8517f492eb128bc7a130
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_radians-9-65e16c7b13de48a5d36793d0c7d35e14
@@ -0,0 +1 @@
+radians(x) - Converts degrees to radians
diff --git a/sql/hive/src/test/resources/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/sql/hive/src/test/resources/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66
deleted file mode 100644
index e0237c0058f55a841f121be33d902fb3817fcb79..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66
+++ /dev/null
@@ -1,4 +0,0 @@
-radians(x) - Converts degrees to radians
-Example:
-  > SELECT radians(90) FROM src LIMIT 1;
-  1.5707963267949mo
diff --git a/sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 b/sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2
index d19c5b50fe5c4a2a3bad536ec65a430723925850..a4de35428cc8f39030dbd24e68db12c0df696287 100644
--- a/sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2
+++ b/sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2
@@ -1 +1 @@
-rand([seed]) - Returns a pseudorandom number between 0 and 1
\ No newline at end of file
+rand([seed]) - Returns a pseudorandom number between 0 and 1
diff --git a/sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e b/sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e
index d19c5b50fe5c4a2a3bad536ec65a430723925850..a4de35428cc8f39030dbd24e68db12c0df696287 100644
--- a/sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e
+++ b/sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e
@@ -1 +1 @@
-rand([seed]) - Returns a pseudorandom number between 0 and 1
\ No newline at end of file
+rand([seed]) - Returns a pseudorandom number between 0 and 1
diff --git a/sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 b/sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1
deleted file mode 100644
index 1d7658151cd62a1515666a82ab840d4943ea948b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1
+++ /dev/null
@@ -1 +0,0 @@
-reflect(class,method[,arg1[,arg2..]]) calls method with reflection
diff --git a/sql/hive/src/test/resources/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee b/sql/hive/src/test/resources/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee
deleted file mode 100644
index ddf986c01e5b131220cf6a2f810108cfe25112e5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee
+++ /dev/null
@@ -1,4 +0,0 @@
-reflect(class,method[,arg1[,arg2..]]) calls method with reflection
-Synonyms: java_method
-Use this UDF to call Java methods by matching the argument signature
-
diff --git a/sql/hive/src/test/resources/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 b/sql/hive/src/test/resources/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 b/sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184
deleted file mode 100644
index cd35e5b290db55467951f6fad9d0e9799048e5c6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184
+++ /dev/null
@@ -1 +0,0 @@
-reflect2(arg0,method[,arg1[,arg2..]]) calls method of arg0 with reflection
diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 b/sql/hive/src/test/resources/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19
deleted file mode 100644
index 48ef97292ab628d83d6ca082faa891c8c2d7b2b9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19
+++ /dev/null
@@ -1,3 +0,0 @@
-reflect2(arg0,method[,arg1[,arg2..]]) calls method of arg0 with reflection
-Use this UDF to call Java methods by matching the argument signature
-
diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca b/sql/hive/src/test/resources/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 b/sql/hive/src/test/resources/golden/udf_regexp-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92
rename to sql/hive/src/test/resources/golden/udf_regexp-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 b/sql/hive/src/test/resources/golden/udf_regexp-1-19917611f74aedc0922560f7f2595948
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948
rename to sql/hive/src/test/resources/golden/udf_regexp-1-19917611f74aedc0922560f7f2595948
diff --git a/sql/hive/src/test/resources/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 b/sql/hive/src/test/resources/golden/udf_regexp-2-f7f0527cd47612d7f256edd5f8963800
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800
rename to sql/hive/src/test/resources/golden/udf_regexp-2-f7f0527cd47612d7f256edd5f8963800
diff --git a/sql/hive/src/test/resources/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 b/sql/hive/src/test/resources/golden/udf_regexp-3-59aff54bae544ee620141e4e629f167a
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11
rename to sql/hive/src/test/resources/golden/udf_regexp-3-59aff54bae544ee620141e4e629f167a
diff --git a/sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 b/sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9
index 429057caf71f096faa57d54c7db68e94659358cd..6f4b3cea94c03bab4023a90773184aa83fd136e5 100644
--- a/sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9
+++ b/sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9
@@ -1 +1 @@
-regexp_extract(str, regexp[, idx]) - extracts a group that matches regexp
\ No newline at end of file
+regexp_extract(str, regexp[, idx]) - extracts a group that matches regexp
diff --git a/sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb b/sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb
index 30e237490811f9a24785b728f34a21552997f4a9..fc9d4aa686db9e41a1761afd314d0c18081afd18 100644
--- a/sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb
+++ b/sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb
@@ -1,4 +1,4 @@
 regexp_extract(str, regexp[, idx]) - extracts a group that matches regexp
 Example:
   > SELECT regexp_extract('100-200', '(\d+)-(\d+)', 1) FROM src LIMIT 1;
-  '100'
\ No newline at end of file
+  '100'
diff --git a/sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e b/sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e
index 1a38701d68875db962cb3e24c303ca693eed6a1d..193093b0edcaa729fe0b3a395963567d00c75891 100644
--- a/sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e
+++ b/sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e
@@ -1 +1 @@
-regexp_replace(str, regexp, rep) - replace all substrings of str that match regexp with rep
\ No newline at end of file
+regexp_replace(str, regexp, rep) - replace all substrings of str that match regexp with rep
diff --git a/sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc b/sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc
index bebfabaf637fb3ea469dc41f1980ae326afe073d..6e3577aba5da25d89122d5f8f5ccc042edf306fa 100644
--- a/sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc
+++ b/sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc
@@ -1,4 +1,4 @@
 regexp_replace(str, regexp, rep) - replace all substrings of str that match regexp with rep
 Example:
   > SELECT regexp_replace('100-200', '(\d+)', 'num') FROM src LIMIT 1;
-  'num-num'
\ No newline at end of file
+  'num-num'
diff --git a/sql/hive/src/test/resources/golden/lock4-4-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_repeat-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/lock4-4-16367c381d4b189b3640c92511244bfe
rename to sql/hive/src/test/resources/golden/udf_repeat-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b b/sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b
deleted file mode 100644
index 694c367436f3c264586a3893ed618d9e97d3c7d3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b
+++ /dev/null
@@ -1 +0,0 @@
-repeat(str, n) - repeat str n times 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b b/sql/hive/src/test/resources/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b
deleted file mode 100644
index 5df19ba1c2cbf5789f1e9bddffdb083fd3f0849a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b
+++ /dev/null
@@ -1,4 +0,0 @@
-repeat(str, n) - repeat str n times 
-Example:
-  > SELECT repeat('123', 2) FROM src LIMIT 1;
-  '123123'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_repeat-1-fdeae3e88f27ef148281d69ac8c4b23b b/sql/hive/src/test/resources/golden/udf_repeat-1-fdeae3e88f27ef148281d69ac8c4b23b
new file mode 100644
index 0000000000000000000000000000000000000000..23a6a30503468eaa5212802c4fc9dafeb58bd861
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_repeat-1-fdeae3e88f27ef148281d69ac8c4b23b
@@ -0,0 +1 @@
+repeat(str, n) - repeat str n times 
diff --git a/sql/hive/src/test/resources/golden/udf_repeat-2-836be47190989d8975a09a545ecbfe0b b/sql/hive/src/test/resources/golden/udf_repeat-2-836be47190989d8975a09a545ecbfe0b
new file mode 100644
index 0000000000000000000000000000000000000000..4f4b491b2807c44fe59b64d5b8a819c3f29e294e
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_repeat-2-836be47190989d8975a09a545ecbfe0b
@@ -0,0 +1,4 @@
+repeat(str, n) - repeat str n times 
+Example:
+  > SELECT repeat('123', 2) FROM src LIMIT 1;
+  '123123'
diff --git a/sql/hive/src/test/resources/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a b/sql/hive/src/test/resources/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d b/sql/hive/src/test/resources/golden/udf_repeat-3-3a3180b4d7c59ee477ce4bebf8e6adec
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d
rename to sql/hive/src/test/resources/golden/udf_repeat-3-3a3180b4d7c59ee477ce4bebf8e6adec
diff --git a/sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 b/sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365
deleted file mode 100644
index 45425cf087c095af015b3dc79a08cbfa0fdc7430..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365
+++ /dev/null
@@ -1 +0,0 @@
-FacebookFacebookFacebook			
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_repeat-4-64c5fce0c5ad4c26680a842aa740dc57 b/sql/hive/src/test/resources/golden/udf_repeat-4-64c5fce0c5ad4c26680a842aa740dc57
new file mode 100644
index 0000000000000000000000000000000000000000..5a355c1c58fc74a93a25f9573c2024b95f74d5c1
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_repeat-4-64c5fce0c5ad4c26680a842aa740dc57
@@ -0,0 +1 @@
+FacebookFacebookFacebook			
diff --git a/sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 b/sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38
index fd9ac3081a1cc3b2a30bc6280cc9b86942c90a8f..068aeb36f647729573e64da256a8f2e61d904b7e 100644
--- a/sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38
+++ b/sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38
@@ -1 +1 @@
-str rlike regexp - Returns true if str matches regexp and false otherwise
\ No newline at end of file
+str rlike regexp - Returns true if str matches regexp and false otherwise
diff --git a/sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca b/sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca
index 43b06945caa5448c528e5e582707f0caf7c23d85..ab346793b9d3d708e9677fabaf127c8bceff85a7 100644
--- a/sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca
+++ b/sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca
@@ -2,4 +2,4 @@ str rlike regexp - Returns true if str matches regexp and false otherwise
 Synonyms: regexp
 Example:
   > SELECT 'fb' rlike '.*' FROM src LIMIT 1;
-  true
\ No newline at end of file
+  true
diff --git a/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 b/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15
deleted file mode 100644
index e4586b2e73a93b003f5be1d2078746dedd33a801..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15
+++ /dev/null
@@ -1 +0,0 @@
-round(x[, d]) - round x to d decimal places
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_round-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7
rename to sql/hive/src/test/resources/golden/udf_round-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_round-1-10b53ca1f15fd7879365926f86512d15 b/sql/hive/src/test/resources/golden/udf_round-1-10b53ca1f15fd7879365926f86512d15
new file mode 100644
index 0000000000000000000000000000000000000000..49fdc0a774e7092c8133d12bd67df832f7a0f2ee
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_round-1-10b53ca1f15fd7879365926f86512d15
@@ -0,0 +1 @@
+round(x[, d]) - round x to d decimal places
diff --git a/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 b/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8
deleted file mode 100644
index c0d5b480e97518b420f6d0d52209a3ea4fad4658..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8
+++ /dev/null
@@ -1,4 +0,0 @@
-round(x[, d]) - round x to d decimal places
-Example:
-  > SELECT round(12.3456, 1) FROM src LIMIT 1;
-  12.3'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_round-2-2367bcc43510dedc80bdb6707e434da8 b/sql/hive/src/test/resources/golden/udf_round-2-2367bcc43510dedc80bdb6707e434da8
new file mode 100644
index 0000000000000000000000000000000000000000..862adeae821ff7d4747ac4ed4dba60981046b289
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_round-2-2367bcc43510dedc80bdb6707e434da8
@@ -0,0 +1,4 @@
+round(x[, d]) - round x to d decimal places
+Example:
+  > SELECT round(12.3456, 1) FROM src LIMIT 1;
+  12.3'
diff --git a/sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 b/sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374
deleted file mode 100644
index 0924e3869076d56ce0834a89a98e293c6e189992..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374
+++ /dev/null
@@ -1 +0,0 @@
-NULL	NULL	NULL	Infinity	NaN
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c b/sql/hive/src/test/resources/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c
deleted file mode 100644
index 3b9c30929a240431cdf2a135bd33e188462c120b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c
+++ /dev/null
@@ -1 +0,0 @@
-55555	55555.0	55555.0	55555.0	55555.0	55560.0	55600.0	56000.0	60000.0	100000.0	0.0	0.0	0.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_round-3-fa4d11da8e1eba258ed191ed5f1447de b/sql/hive/src/test/resources/golden/udf_round-3-fa4d11da8e1eba258ed191ed5f1447de
new file mode 100644
index 0000000000000000000000000000000000000000..fc6e4224259d8cf697db854f75435228c5e2b8ca
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_round-3-fa4d11da8e1eba258ed191ed5f1447de
@@ -0,0 +1 @@
+NULL	NULL	NULL	NULL	NaN
diff --git a/sql/hive/src/test/resources/golden/udf_round-4-b87ccaa1e0a87c558b56d59a8a074396 b/sql/hive/src/test/resources/golden/udf_round-4-b87ccaa1e0a87c558b56d59a8a074396
new file mode 100644
index 0000000000000000000000000000000000000000..f8d833cc0880edd7d32d85664442cb443df7ae32
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_round-4-b87ccaa1e0a87c558b56d59a8a074396
@@ -0,0 +1 @@
+55555	55555	55555	55555	55555	55560	55600	56000	60000	100000	0	0	0
diff --git a/sql/hive/src/test/resources/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 b/sql/hive/src/test/resources/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4
deleted file mode 100644
index 1f243f6cbc6dbcba95ca0e43d2d058c149b3bad4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4
+++ /dev/null
@@ -1 +0,0 @@
-125.0	125.0	125.3	125.32	125.315	125.315	130.0	100.0	0.0	0.0	-125.0	-125.0	-125.3	-125.32	-125.315	-125.315	-130.0	-100.0	0.0	0.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_round-5-441d0075081ae87579c959d714c4922d b/sql/hive/src/test/resources/golden/udf_round-5-441d0075081ae87579c959d714c4922d
new file mode 100644
index 0000000000000000000000000000000000000000..389ab6417f19db3a5a16fd8edaa515f569b4db86
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_round-5-441d0075081ae87579c959d714c4922d
@@ -0,0 +1 @@
+125.0	125.0	125.3	125.32	125.315	125.315	130.0	100.0	0.0	0.0	-125.0	-125.0	-125.3	-125.32	-125.315	-125.315	-130.0	-100.0	0.0	0.0
diff --git a/sql/hive/src/test/resources/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a b/sql/hive/src/test/resources/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a
deleted file mode 100644
index 918404cec804756a0a686d3d8fdff37a9c7581c0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a
+++ /dev/null
@@ -1 +0,0 @@
-0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	3.0	3.1	3.14	3.142	3.1416	3.14159	3.141593	3.1415927	3.14159265	3.141592654	3.1415926536	3.14159265359	3.14159265359	3.1415926535898	3.1415926535898	3.14159265358979	3.141592653589793	3.141592653589793
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_round-6-4658ec3bc034b43e0477bf2474939449 b/sql/hive/src/test/resources/golden/udf_round-6-4658ec3bc034b43e0477bf2474939449
new file mode 100644
index 0000000000000000000000000000000000000000..3b083b3e26c4a54171c8ec224f0108204162daff
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_round-6-4658ec3bc034b43e0477bf2474939449
@@ -0,0 +1 @@
+0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	0.0	3.0	3.1	3.14	3.142	3.1416	3.14159	3.141593	3.1415927	3.14159265	3.141592654	3.1415926536	3.14159265359	3.14159265359	3.1415926535898	3.1415926535898	3.14159265358979	3.141592653589793	3.141592653589793
diff --git a/sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b b/sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b
deleted file mode 100644
index af105563af144b2390db30e2cb904a04ea75a75c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b
+++ /dev/null
@@ -1 +0,0 @@
-1809242.315111134	-1809242.315111134
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_round-7-74ff5a4862c80bd8fd84bede1a0320d b/sql/hive/src/test/resources/golden/udf_round-7-74ff5a4862c80bd8fd84bede1a0320d
new file mode 100644
index 0000000000000000000000000000000000000000..c3496bf5b6d1bbbc8816a3adce0faead5ad9ec27
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_round-7-74ff5a4862c80bd8fd84bede1a0320d
@@ -0,0 +1 @@
+1809242.315111134	-1809242.315111134	1809242.315111134	-1809242.315111134
diff --git a/sql/hive/src/test/resources/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 b/sql/hive/src/test/resources/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce b/sql/hive/src/test/resources/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 b/sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298
deleted file mode 100644
index b955cad2a39a09a1c4143ccffb078ac94bb80b4c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298
+++ /dev/null
@@ -1 +0,0 @@
-NaN
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 b/sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93
deleted file mode 100644
index b955cad2a39a09a1c4143ccffb078ac94bb80b4c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93
+++ /dev/null
@@ -1 +0,0 @@
-NaN
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 b/sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3
deleted file mode 100644
index b955cad2a39a09a1c4143ccffb078ac94bb80b4c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3
+++ /dev/null
@@ -1 +0,0 @@
-NaN
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b b/sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b
deleted file mode 100644
index f40e633f703c9b28ea5b0662edfc707c2aace180..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b
+++ /dev/null
@@ -1 +0,0 @@
-Infinity	Infinity	Infinity	Infinity
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/udf_round_3-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a
rename to sql/hive/src/test/resources/golden/udf_round_3-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c b/sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c
deleted file mode 100644
index 3714de0db18dc16406fd8eb7eb8fbd1902b91973..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c
+++ /dev/null
@@ -1 +0,0 @@
--128	127	0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_round_3-1-15a7f123f596e28e6f238063ba4e3d6d b/sql/hive/src/test/resources/golden/udf_round_3-1-15a7f123f596e28e6f238063ba4e3d6d
new file mode 100644
index 0000000000000000000000000000000000000000..0a104d81ef51ead9c3dc2e0720838401ce5d4c83
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_round_3-1-15a7f123f596e28e6f238063ba4e3d6d
@@ -0,0 +1 @@
+-128	127	0
diff --git a/sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 b/sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0
deleted file mode 100644
index a9265d7b8a1b799cd615152c694d8143958c606c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0
+++ /dev/null
@@ -1 +0,0 @@
--32768	32767	-129	128
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 b/sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10
deleted file mode 100644
index 2d25f54073df71ed8a8bcf15c3a30b7f2fde3a40..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10
+++ /dev/null
@@ -1 +0,0 @@
--2147483648	2147483647	-32769	32768
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_round_3-2-a03bf4e99027d4814a32c84d89d42cca b/sql/hive/src/test/resources/golden/udf_round_3-2-a03bf4e99027d4814a32c84d89d42cca
new file mode 100644
index 0000000000000000000000000000000000000000..972dee75fae8ad2dbea01fb618aeeb619fc2ff16
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_round_3-2-a03bf4e99027d4814a32c84d89d42cca
@@ -0,0 +1 @@
+-32768	32767	-129	128
diff --git a/sql/hive/src/test/resources/golden/udf_round_3-3-e3c5b35d67ef3de2800a1836718e8ac9 b/sql/hive/src/test/resources/golden/udf_round_3-3-e3c5b35d67ef3de2800a1836718e8ac9
new file mode 100644
index 0000000000000000000000000000000000000000..50928a309cf3aeadd80ae5609da24874047e0be2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_round_3-3-e3c5b35d67ef3de2800a1836718e8ac9
@@ -0,0 +1 @@
+-2147483648	2147483647	-32769	32768
diff --git a/sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 b/sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9
deleted file mode 100644
index 41a2624c6cfeb65f3b5f522c952fce12a2f6cbd0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9
+++ /dev/null
@@ -1 +0,0 @@
--9223372036854775808	9223372036854775807	-2147483649	2147483648
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_round_3-4-8449fbdabbb4b1e6beab89be0af498f2 b/sql/hive/src/test/resources/golden/udf_round_3-4-8449fbdabbb4b1e6beab89be0af498f2
new file mode 100644
index 0000000000000000000000000000000000000000..c9e55e0ec0b952d36d28c3a1f3ec0ba48eb40482
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_round_3-4-8449fbdabbb4b1e6beab89be0af498f2
@@ -0,0 +1 @@
+-9223372036854775808	9223372036854775807	-2147483649	2147483648
diff --git a/sql/hive/src/test/resources/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 b/sql/hive/src/test/resources/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7
deleted file mode 100644
index 98d3f53dfc4421b1e921d72df608bbd3635318c5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7
+++ /dev/null
@@ -1 +0,0 @@
-126.0	127.0	32766.0	32767.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_round_3-5-3844531c8cff115d6a33636db0a26ad b/sql/hive/src/test/resources/golden/udf_round_3-5-3844531c8cff115d6a33636db0a26ad
new file mode 100644
index 0000000000000000000000000000000000000000..3898a62cae1b5b268d3fd611eae11c819b71fe5a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_round_3-5-3844531c8cff115d6a33636db0a26ad
@@ -0,0 +1 @@
+126.0	127.0	32766.0	32767.0
diff --git a/sql/hive/src/test/resources/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/udf_rpad-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027
rename to sql/hive/src/test/resources/golden/udf_rpad-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c b/sql/hive/src/test/resources/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c
deleted file mode 100644
index 7cb2d71d4b80db70da19e001cb67da102c41446c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c
+++ /dev/null
@@ -1 +0,0 @@
-rpad(str, len, pad) - Returns str, right-padded with pad to a length of len
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 b/sql/hive/src/test/resources/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0
deleted file mode 100644
index 2b198b8dc96c4a2e1dfb0d9e46e6c30d93623b77..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0
+++ /dev/null
@@ -1,6 +0,0 @@
-rpad(str, len, pad) - Returns str, right-padded with pad to a length of len
-If str is longer than len, the return value is shortened to len characters.
-Example:
-  > SELECT rpad('hi', 5, '??') FROM src LIMIT 1;
-  'hi???'  > SELECT rpad('hi', 1, '??') FROM src LIMIT 1;
-  'h'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_rpad-1-56de896c5fe8b40b22a9ed55ed79889c b/sql/hive/src/test/resources/golden/udf_rpad-1-56de896c5fe8b40b22a9ed55ed79889c
new file mode 100644
index 0000000000000000000000000000000000000000..f451030a3a142488360a060cee0ce151f538c54d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_rpad-1-56de896c5fe8b40b22a9ed55ed79889c
@@ -0,0 +1 @@
+rpad(str, len, pad) - Returns str, right-padded with pad to a length of len
diff --git a/sql/hive/src/test/resources/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 b/sql/hive/src/test/resources/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_rpad-2-48d17e5d5d6188300d048f987fab2ca0 b/sql/hive/src/test/resources/golden/udf_rpad-2-48d17e5d5d6188300d048f987fab2ca0
new file mode 100644
index 0000000000000000000000000000000000000000..67dcf2427362d0e0b49178bdf11ea98c6c3489c2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_rpad-2-48d17e5d5d6188300d048f987fab2ca0
@@ -0,0 +1,6 @@
+rpad(str, len, pad) - Returns str, right-padded with pad to a length of len
+If str is longer than len, the return value is shortened to len characters.
+Example:
+  > SELECT rpad('hi', 5, '??') FROM src LIMIT 1;
+  'hi???'  > SELECT rpad('hi', 1, '??') FROM src LIMIT 1;
+  'h'
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b b/sql/hive/src/test/resources/golden/udf_rpad-3-66acb969c28a8e376782ccd0d442b450
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b
rename to sql/hive/src/test/resources/golden/udf_rpad-3-66acb969c28a8e376782ccd0d442b450
diff --git a/sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 b/sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871
deleted file mode 100644
index b2645e365c8a98fb7099b1fbb1502387089fb2ba..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871
+++ /dev/null
@@ -1 +0,0 @@
-h	hi...	hi1231
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_rpad-4-299dee5a72aad2a2738d7841a89bb71b b/sql/hive/src/test/resources/golden/udf_rpad-4-299dee5a72aad2a2738d7841a89bb71b
new file mode 100644
index 0000000000000000000000000000000000000000..0d73ca82abf898ab634a30ce7e9e6b00a928ffad
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_rpad-4-299dee5a72aad2a2738d7841a89bb71b
@@ -0,0 +1 @@
+h	hi...	hi1231
diff --git a/sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a b/sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a
index 94d6aeed8f53309726490ba6536db70276c83873..d0d2416d7ee9071e0ba7717ce1836d2cbe699bac 100644
--- a/sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a
+++ b/sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a
@@ -1 +1 @@
-rtrim(str) - Removes the trailing space characters from str 
\ No newline at end of file
+rtrim(str) - Removes the trailing space characters from str 
diff --git a/sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd b/sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd
index 957e608a7c7325d4e8136bb4a4bb832b5f710ec6..697cdcc8198b3a95d6003330d37382be526c378d 100644
--- a/sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd
+++ b/sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd
@@ -1,4 +1,4 @@
 rtrim(str) - Removes the trailing space characters from str 
 Example:
   > SELECT rtrim('facebook   ') FROM src LIMIT 1;
-  'facebook'
\ No newline at end of file
+  'facebook'
diff --git a/sql/hive/src/test/resources/golden/merge4-2-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/udf_second-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/merge4-2-43d53504df013e6b35f81811138a167a
rename to sql/hive/src/test/resources/golden/udf_second-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 b/sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2
deleted file mode 100644
index 577c90254cb5a29fe2a98bb0bd611b0dc8c93447..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2
+++ /dev/null
@@ -1 +0,0 @@
-second(date) - Returns the second of date
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 b/sql/hive/src/test/resources/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888
deleted file mode 100644
index 4b48294e5b9add346ba53356b92e2589b72054e4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888
+++ /dev/null
@@ -1,7 +0,0 @@
-second(date) - Returns the second of date
-date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'.
-Example:
-   > SELECT second('2009-07-30 12:58:59') FROM src LIMIT 1;
-  59
-  > SELECT second('12:58:59') FROM src LIMIT 1;
-  59
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_second-1-e004a6f20da3fa6db868ec847b217ff2 b/sql/hive/src/test/resources/golden/udf_second-1-e004a6f20da3fa6db868ec847b217ff2
new file mode 100644
index 0000000000000000000000000000000000000000..ad2b24b8eb11cfae9ba4ad0c93c45d54fb4111f5
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_second-1-e004a6f20da3fa6db868ec847b217ff2
@@ -0,0 +1 @@
+second(date) - Returns the second of date
diff --git a/sql/hive/src/test/resources/golden/udf_second-2-3525f55f4f13253c42b3abaa53d77888 b/sql/hive/src/test/resources/golden/udf_second-2-3525f55f4f13253c42b3abaa53d77888
new file mode 100644
index 0000000000000000000000000000000000000000..9dc38c3e79129da6859b5964eebe180405d166ed
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_second-2-3525f55f4f13253c42b3abaa53d77888
@@ -0,0 +1,7 @@
+second(date) - Returns the second of date
+date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'.
+Example:
+   > SELECT second('2009-07-30 12:58:59') FROM src LIMIT 1;
+  59
+  > SELECT second('12:58:59') FROM src LIMIT 1;
+  59
diff --git a/sql/hive/src/test/resources/golden/udf_second-2-d678372e3837a16be245d2e33482f17f b/sql/hive/src/test/resources/golden/udf_second-2-d678372e3837a16be245d2e33482f17f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf b/sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf
deleted file mode 100644
index 4d5ef5cf4a699d542ec9a5a49dc6faf080eca3a8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf
+++ /dev/null
@@ -1 +0,0 @@
-15	15	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 b/sql/hive/src/test/resources/golden/udf_second-3-d678372e3837a16be245d2e33482f17f
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67
rename to sql/hive/src/test/resources/golden/udf_second-3-d678372e3837a16be245d2e33482f17f
diff --git a/sql/hive/src/test/resources/golden/udf_second-4-2496e4d3c64ca028184431c2930d82cf b/sql/hive/src/test/resources/golden/udf_second-4-2496e4d3c64ca028184431c2930d82cf
new file mode 100644
index 0000000000000000000000000000000000000000..3ddfab9c754f4aabb483f3456e0959e947eae653
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_second-4-2496e4d3c64ca028184431c2930d82cf
@@ -0,0 +1 @@
+15	15	NULL
diff --git a/sql/hive/src/test/resources/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f b/sql/hive/src/test/resources/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 b/sql/hive/src/test/resources/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 b/sql/hive/src/test/resources/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 b/sql/hive/src/test/resources/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80
deleted file mode 100644
index fea4f860c8465d8b0a87160874a93d9f3d15a9c3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80
+++ /dev/null
@@ -1,31 +0,0 @@
-41626672616765
-446174656E
-48697665
-496E74657270756E6B74696F6E
-4D756C7469706C65
-53C3A4747A65
-554446
-5665727765636873656C756E67
-5765726B7A657567
-616C73
-61757367657A656963686E65746573
-646965
-646965
-6469657365
-646F6368
-65696E
-66756E6B74696F6E69657274
-66C3BC72
-676562696C646574656E
-696D6D6572
-697374
-697374
-6D61736368696E656C6C65
-6E6F6368
-7363686C65636874
-756E64
-756E64
-7669656C6C6569636874
-7669656C7365697469676572
-766F6E
-C39C6265727365747A756E67
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 b/sql/hive/src/test/resources/golden/udf_sentences-12-d55c04a079ca97402509868f24921685
deleted file mode 100644
index c49ca6bb5a1b0a639767bf8a3047c72f1995b4a1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sentences-12-d55c04a079ca97402509868f24921685
+++ /dev/null
@@ -1 +0,0 @@
-[["Hive","is","an","excellent","tool","for","data","querying","and","perhaps","more","versatile","than","machine","translation"],["Multiple","ill-formed","sentences","confounding","punctuation","and","yet","this","UDF","still","works"]]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 b/sql/hive/src/test/resources/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 b/sql/hive/src/test/resources/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 b/sql/hive/src/test/resources/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80
deleted file mode 100644
index b798628e56686637b1c18e095e21799e286b3d06..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80
+++ /dev/null
@@ -1,50 +0,0 @@
-48697665
-48697665
-554446
-6175746F6D617469717565
-6175746F6D617469717565
-6365
-636F6E667573696F6E
-6465
-6465
-646573
-646F6E6EC3A96573
-646F6E6EC3A96573
-656E636F7265
-657374
-657374
-6574
-6574
-6574
-657863656C6C656E74
-657863656C6C656E74
-666F6E6374696F6E6E65
-666F726DC3A96573
-6C61
-6C61
-6C61
-6C6573
-6C6573
-6D616C
-6D756C7469706C6573
-6F7574696C
-6F7574696C
-706575742DC3AA747265
-706575742DC3AA747265
-70687261736573
-706C7573
-706C7573
-706F6C7976616C656E74
-706F6C7976616C656E74
-706F6E6374756174696F6E
-706F7572
-706F7572
-706F757274616E74
-717565
-717565
-72657175C3AA746573
-72657175C3AA746573
-74726164756374696F6E
-74726164756374696F6E
-756E
-756E
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 b/sql/hive/src/test/resources/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de b/sql/hive/src/test/resources/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f b/sql/hive/src/test/resources/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 b/sql/hive/src/test/resources/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 b/sql/hive/src/test/resources/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e b/sql/hive/src/test/resources/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_sign-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7
rename to sql/hive/src/test/resources/golden/udf_sign-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c b/sql/hive/src/test/resources/golden/udf_sign-1-cddd6ec2a7dfc2f8f7e35bc39df541f9
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c
rename to sql/hive/src/test/resources/golden/udf_sign-1-cddd6ec2a7dfc2f8f7e35bc39df541f9
diff --git a/sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 b/sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159
deleted file mode 100644
index 171538eb0b00f4eddffa17929796de55b838f34b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159
+++ /dev/null
@@ -1 +0,0 @@
-0.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a b/sql/hive/src/test/resources/golden/udf_sign-10-9a5326b0bf612fed4ce0b04770bebc16
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a
rename to sql/hive/src/test/resources/golden/udf_sign-10-9a5326b0bf612fed4ce0b04770bebc16
diff --git a/sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa b/sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa
deleted file mode 100644
index 3345fbb8f2c3595109892df0504c7533f79ba1a6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa
+++ /dev/null
@@ -1 +0,0 @@
-sign(x) - returns the sign of x )
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a b/sql/hive/src/test/resources/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a
deleted file mode 100644
index bdacec8810e77172c514245c7b06e33f78c3b1b3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a
+++ /dev/null
@@ -1,4 +0,0 @@
-sign(x) - returns the sign of x )
-Example:
-   > SELECT sign(40) FROM src LIMIT 1;
-  1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_sign-11-fc7341f89c3cd3c43e436242d8aa61fa b/sql/hive/src/test/resources/golden/udf_sign-11-fc7341f89c3cd3c43e436242d8aa61fa
new file mode 100644
index 0000000000000000000000000000000000000000..60533947bcfb95c645172c62616cf3e661e78684
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_sign-11-fc7341f89c3cd3c43e436242d8aa61fa
@@ -0,0 +1 @@
+sign(x) - returns the sign of x )
diff --git a/sql/hive/src/test/resources/golden/udf_sign-12-74237f5ecc497813cf9738b21647847a b/sql/hive/src/test/resources/golden/udf_sign-12-74237f5ecc497813cf9738b21647847a
new file mode 100644
index 0000000000000000000000000000000000000000..d888f9e9cda191bf965e6fb5ae6961de85b7bfc7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_sign-12-74237f5ecc497813cf9738b21647847a
@@ -0,0 +1,4 @@
+sign(x) - returns the sign of x )
+Example:
+   > SELECT sign(40) FROM src LIMIT 1;
+  1
diff --git a/sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef b/sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef
deleted file mode 100644
index 31a5b0b81dc51e9dbab9c4b9cbbe46eb388ced71..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef
+++ /dev/null
@@ -1 +0,0 @@
--1.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_sign-2-fba3eb5c16eca01b0c0f5918dbbffbc7 b/sql/hive/src/test/resources/golden/udf_sign-2-fba3eb5c16eca01b0c0f5918dbbffbc7
new file mode 100644
index 0000000000000000000000000000000000000000..ba66466c2a0d0a1080ad71d8e1a67a97f336c384
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_sign-2-fba3eb5c16eca01b0c0f5918dbbffbc7
@@ -0,0 +1 @@
+0.0
diff --git a/sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d b/sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d
deleted file mode 100644
index 9f8e9b69a33f4e8067d5b21661a35d8856758aba..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d
+++ /dev/null
@@ -1 +0,0 @@
-1.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_sign-3-d3e4489fc6873b5dbc4fe3e99ef13900 b/sql/hive/src/test/resources/golden/udf_sign-3-d3e4489fc6873b5dbc4fe3e99ef13900
new file mode 100644
index 0000000000000000000000000000000000000000..18e16e38c5de68635347d560e655fada9acc0fab
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_sign-3-d3e4489fc6873b5dbc4fe3e99ef13900
@@ -0,0 +1 @@
+-1.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c b/sql/hive/src/test/resources/golden/udf_sign-4-9a5326b0bf612fed4ce0b04770bebc16
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c
rename to sql/hive/src/test/resources/golden/udf_sign-4-9a5326b0bf612fed4ce0b04770bebc16
diff --git a/sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa b/sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa
deleted file mode 100644
index 3345fbb8f2c3595109892df0504c7533f79ba1a6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa
+++ /dev/null
@@ -1 +0,0 @@
-sign(x) - returns the sign of x )
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a b/sql/hive/src/test/resources/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a
deleted file mode 100644
index bdacec8810e77172c514245c7b06e33f78c3b1b3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a
+++ /dev/null
@@ -1,4 +0,0 @@
-sign(x) - returns the sign of x )
-Example:
-   > SELECT sign(40) FROM src LIMIT 1;
-  1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_sign-5-fc7341f89c3cd3c43e436242d8aa61fa b/sql/hive/src/test/resources/golden/udf_sign-5-fc7341f89c3cd3c43e436242d8aa61fa
new file mode 100644
index 0000000000000000000000000000000000000000..60533947bcfb95c645172c62616cf3e661e78684
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_sign-5-fc7341f89c3cd3c43e436242d8aa61fa
@@ -0,0 +1 @@
+sign(x) - returns the sign of x )
diff --git a/sql/hive/src/test/resources/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e b/sql/hive/src/test/resources/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_sign-6-74237f5ecc497813cf9738b21647847a b/sql/hive/src/test/resources/golden/udf_sign-6-74237f5ecc497813cf9738b21647847a
new file mode 100644
index 0000000000000000000000000000000000000000..d888f9e9cda191bf965e6fb5ae6961de85b7bfc7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_sign-6-74237f5ecc497813cf9738b21647847a
@@ -0,0 +1,4 @@
+sign(x) - returns the sign of x )
+Example:
+   > SELECT sign(40) FROM src LIMIT 1;
+  1
diff --git a/sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 b/sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159
deleted file mode 100644
index 171538eb0b00f4eddffa17929796de55b838f34b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159
+++ /dev/null
@@ -1 +0,0 @@
-0.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a b/sql/hive/src/test/resources/golden/udf_sign-7-ed2aaa1a416c0cccc04de970424e1860
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a
rename to sql/hive/src/test/resources/golden/udf_sign-7-ed2aaa1a416c0cccc04de970424e1860
diff --git a/sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef b/sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef
deleted file mode 100644
index 31a5b0b81dc51e9dbab9c4b9cbbe46eb388ced71..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef
+++ /dev/null
@@ -1 +0,0 @@
--1.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_sign-8-fba3eb5c16eca01b0c0f5918dbbffbc7 b/sql/hive/src/test/resources/golden/udf_sign-8-fba3eb5c16eca01b0c0f5918dbbffbc7
new file mode 100644
index 0000000000000000000000000000000000000000..ba66466c2a0d0a1080ad71d8e1a67a97f336c384
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_sign-8-fba3eb5c16eca01b0c0f5918dbbffbc7
@@ -0,0 +1 @@
+0.0
diff --git a/sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d b/sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d
deleted file mode 100644
index 9f8e9b69a33f4e8067d5b21661a35d8856758aba..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d
+++ /dev/null
@@ -1 +0,0 @@
-1.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_sign-9-d3e4489fc6873b5dbc4fe3e99ef13900 b/sql/hive/src/test/resources/golden/udf_sign-9-d3e4489fc6873b5dbc4fe3e99ef13900
new file mode 100644
index 0000000000000000000000000000000000000000..18e16e38c5de68635347d560e655fada9acc0fab
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_sign-9-d3e4489fc6873b5dbc4fe3e99ef13900
@@ -0,0 +1 @@
+-1.0
diff --git a/sql/hive/src/test/resources/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee b/sql/hive/src/test/resources/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee
deleted file mode 100644
index 6155d9a2eae5573ec6b40c14cfb0123a1860ed37..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee
+++ /dev/null
@@ -1 +0,0 @@
-sin(x) - returns the sine of x (x is in radians)
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/merge4-4-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_sin-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/merge4-4-16367c381d4b189b3640c92511244bfe
rename to sql/hive/src/test/resources/golden/udf_sin-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 b/sql/hive/src/test/resources/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63
deleted file mode 100644
index a39ed8840b916a5379dfeaa266b205f5c8ab20fe..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63
+++ /dev/null
@@ -1,4 +0,0 @@
-sin(x) - returns the sine of x (x is in radians)
-Example:
-   > SELECT sin(0) FROM src LIMIT 1;
-  0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_sin-1-40b50393869eb0bcde66e36fe41078ee b/sql/hive/src/test/resources/golden/udf_sin-1-40b50393869eb0bcde66e36fe41078ee
new file mode 100644
index 0000000000000000000000000000000000000000..86bbf99dbf06746f5f6653b6b1aa5aa332643616
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_sin-1-40b50393869eb0bcde66e36fe41078ee
@@ -0,0 +1 @@
+sin(x) - returns the sine of x (x is in radians)
diff --git a/sql/hive/src/test/resources/golden/udf_sin-2-2f867f432fb322e21dce353d7eb50c63 b/sql/hive/src/test/resources/golden/udf_sin-2-2f867f432fb322e21dce353d7eb50c63
new file mode 100644
index 0000000000000000000000000000000000000000..0efbed397abf9e93a3a3ddfc2864d9620e3fb599
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_sin-2-2f867f432fb322e21dce353d7eb50c63
@@ -0,0 +1,4 @@
+sin(x) - returns the sine of x (x is in radians)
+Example:
+   > SELECT sin(0) FROM src LIMIT 1;
+  0
diff --git a/sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab b/sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2 b/sql/hive/src/test/resources/golden/udf_sin-3-1d3a615e3aa252a317daa601811820b1
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2
rename to sql/hive/src/test/resources/golden/udf_sin-3-1d3a615e3aa252a317daa601811820b1
diff --git a/sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 b/sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3
deleted file mode 100644
index ee21925b0ccc5ee735c7c8bd27753c019e36d14e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3
+++ /dev/null
@@ -1 +0,0 @@
-0.8304973704919705	0.9999996829318346	-0.479425538604203
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_sin-4-951fb8d311f52ab59d3bacd37d3e611a b/sql/hive/src/test/resources/golden/udf_sin-4-951fb8d311f52ab59d3bacd37d3e611a
new file mode 100644
index 0000000000000000000000000000000000000000..4ca4af756468c8086b12deb3da16c5509d2315d7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_sin-4-951fb8d311f52ab59d3bacd37d3e611a
@@ -0,0 +1 @@
+0.8304973704919705	0.9999996829318346	-0.479425538604203
diff --git a/sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 b/sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62
deleted file mode 100644
index d8952629349a960821af035f1d775385abc6be3e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62
+++ /dev/null
@@ -1 +0,0 @@
-size(a) - Returns the size of a
diff --git a/sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 b/sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8
deleted file mode 100644
index d8952629349a960821af035f1d775385abc6be3e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8
+++ /dev/null
@@ -1 +0,0 @@
-size(a) - Returns the size of a
diff --git a/sql/hive/src/test/resources/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 b/sql/hive/src/test/resources/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd b/sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd
deleted file mode 100644
index 08a708b0902829a41120be2aa2e7fbb92e55ffe4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd
+++ /dev/null
@@ -1 +0,0 @@
-3	1	1	-1
diff --git a/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 b/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042
index 572ecb0462eb7eb7d672da66c71efb73e9eea820..132d6138e4983463bdb7d8dd5fbaa649c6e814f7 100644
--- a/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042
+++ b/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042
@@ -1 +1 @@
-There is no documentation for function 'smallint'
\ No newline at end of file
+There is no documentation for function 'smallint'
diff --git a/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 b/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6
index 572ecb0462eb7eb7d672da66c71efb73e9eea820..132d6138e4983463bdb7d8dd5fbaa649c6e814f7 100644
--- a/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6
+++ b/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6
@@ -1 +1 @@
-There is no documentation for function 'smallint'
\ No newline at end of file
+There is no documentation for function 'smallint'
diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 b/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82
deleted file mode 100644
index d514df4191b8936e18efc30a8fccc444069967d4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82
+++ /dev/null
@@ -1 +0,0 @@
-sort_array(array(obj1, obj2,...)) - Sorts the input array in ascending order according to the natural ordering of the array elements.
diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 b/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93
deleted file mode 100644
index 43e36513de881d11339c1ca66cabe9084ba85213..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93
+++ /dev/null
@@ -1,4 +0,0 @@
-sort_array(array(obj1, obj2,...)) - Sorts the input array in ascending order according to the natural ordering of the array elements.
-Example:
-  > SELECT sort_array(array('b', 'd', 'c', 'a')) FROM src LIMIT 1;
-  'a', 'b', 'c', 'd'
diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de b/sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 b/sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44
deleted file mode 100644
index 2e9458debfd0f16bc34f256df2298ee11aa3cd62..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44
+++ /dev/null
@@ -1 +0,0 @@
-["a","b","c","d","e","f","g"]
diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e b/sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e
deleted file mode 100644
index 94f18d09863a7902588635b93ee36c27ea9d8c24..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e
+++ /dev/null
@@ -1 +0,0 @@
-["enterprise databases","hadoop distributed file system","hadoop map-reduce"]
diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 b/sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3
deleted file mode 100644
index e1968ef44a2ebb2e23746c55cc03f1509a42c41c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3
+++ /dev/null
@@ -1 +0,0 @@
-[1,2,3,4,5,6,7,8,9]
diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 b/sql/hive/src/test/resources/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8
deleted file mode 100644
index b6c12ace4162ba92d0f441b47b8b8752b50e35b1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8
+++ /dev/null
@@ -1 +0,0 @@
-[-3.445,0.777,1.0,1.325,2.003,2.333,9.0]
diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 b/sql/hive/src/test/resources/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a b/sql/hive/src/test/resources/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 b/sql/hive/src/test/resources/golden/udf_space-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644
rename to sql/hive/src/test/resources/golden/udf_space-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 b/sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010
deleted file mode 100644
index ed5bda787df23d04e78ac8dfaa68155fd5ca6cd0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010
+++ /dev/null
@@ -1 +0,0 @@
-space(n) - returns n spaces
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_space-1-91e879c9f01d90eab7bf12fcef256010 b/sql/hive/src/test/resources/golden/udf_space-1-91e879c9f01d90eab7bf12fcef256010
new file mode 100644
index 0000000000000000000000000000000000000000..a443bc3cbf0bdc094c72120e1acf7d09e26fba8d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_space-1-91e879c9f01d90eab7bf12fcef256010
@@ -0,0 +1 @@
+space(n) - returns n spaces
diff --git a/sql/hive/src/test/resources/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 b/sql/hive/src/test/resources/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754
deleted file mode 100644
index 5713d4b0464be295d77fa22b09ee9fbc89d26bb7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754
+++ /dev/null
@@ -1,4 +0,0 @@
-space(n) - returns n spaces
-Example:
-   > SELECT space(2) FROM src LIMIT 1;
-  '  '
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 b/sql/hive/src/test/resources/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_space-2-e4eaf5e96807e122548cb43be9a26754 b/sql/hive/src/test/resources/golden/udf_space-2-e4eaf5e96807e122548cb43be9a26754
new file mode 100644
index 0000000000000000000000000000000000000000..25a7583ef01e9a1291ea5cbaf8e68ad6ea334c6e
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_space-2-e4eaf5e96807e122548cb43be9a26754
@@ -0,0 +1,4 @@
+space(n) - returns n spaces
+Example:
+   > SELECT space(2) FROM src LIMIT 1;
+  '  '
diff --git a/sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 b/sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956
deleted file mode 100644
index 85a16b2abe5ef7a871abf76fdef63dd9ad307397..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956
+++ /dev/null
@@ -1 +0,0 @@
-10	0	1	0	0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 b/sql/hive/src/test/resources/golden/udf_space-3-a1b9dad63547f7ba73a5230d650983b0
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648
rename to sql/hive/src/test/resources/golden/udf_space-3-a1b9dad63547f7ba73a5230d650983b0
diff --git a/sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 b/sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5
deleted file mode 100644
index 8dfaf2745f666e29184f4e8a8b92e40bd596fc73..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5
+++ /dev/null
@@ -1 +0,0 @@
-          		 		
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_space-4-d9de5746edd753507c3f054e1bee7702 b/sql/hive/src/test/resources/golden/udf_space-4-d9de5746edd753507c3f054e1bee7702
new file mode 100644
index 0000000000000000000000000000000000000000..6f07be9b1d0430a03e919c2005489dbb7b38cfb3
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_space-4-d9de5746edd753507c3f054e1bee7702
@@ -0,0 +1 @@
+10	0	1	0	0
diff --git a/sql/hive/src/test/resources/golden/udf_space-5-ce5288dcc60f9412109930bd56752a65 b/sql/hive/src/test/resources/golden/udf_space-5-ce5288dcc60f9412109930bd56752a65
new file mode 100644
index 0000000000000000000000000000000000000000..8f243e851f12fb8777816249a68a5ed5569a7862
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_space-5-ce5288dcc60f9412109930bd56752a65
@@ -0,0 +1 @@
+          		 		
diff --git a/sql/hive/src/test/resources/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 b/sql/hive/src/test/resources/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338
deleted file mode 100644
index 6b183ccfb17b4db472b30d256df249e31a91be10..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338
+++ /dev/null
@@ -1 +0,0 @@
-split(str, regex) - Splits str around occurances that match regex
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 b/sql/hive/src/test/resources/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1
deleted file mode 100644
index 9e5c522da155b9e41ea6dd9c4d2b5e69d0252069..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1
+++ /dev/null
@@ -1,4 +0,0 @@
-split(str, regex) - Splits str around occurances that match regex
-Example:
-  > SELECT split('oneAtwoBthreeC', '[ABC]') FROM src LIMIT 1;
-  ["one", "two", "three"]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 b/sql/hive/src/test/resources/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 b/sql/hive/src/test/resources/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23
deleted file mode 100644
index 9174f1a92557e618a4b063f9f1ee63ec00f80832..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23
+++ /dev/null
@@ -1 +0,0 @@
-["a","b","c"]	["one","two","three"]	[]	["5","4","1","2"]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 b/sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240
index eadd6eeb8b1498e3a69449da5d57686695596e9a..01efc31b30ede56b027d645e0939a5760743dca4 100644
--- a/sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240
+++ b/sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240
@@ -1 +1 @@
-sqrt(x) - returns the square root of x
\ No newline at end of file
+sqrt(x) - returns the square root of x
diff --git a/sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 b/sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77
index 74df32f2212784240c9ad32e59704c52a899117a..16b77c4130fb6827ed6cb3f3c00d215112aef2d4 100644
--- a/sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77
+++ b/sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77
@@ -1,4 +1,4 @@
 sqrt(x) - returns the square root of x
 Example:
    > SELECT sqrt(4) FROM src LIMIT 1;
-  2
\ No newline at end of file
+  2
diff --git a/sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b b/sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b
index 2966c01c1b7b231c2cbfcbd447c1d8dc950d3ed6..5cedcfd415c8856a7a49d0157e966c9e469530ff 100644
--- a/sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b
+++ b/sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b
@@ -1 +1 @@
-std(x) - Returns the standard deviation of a set of numbers
\ No newline at end of file
+std(x) - Returns the standard deviation of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 b/sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027
index 388eaa1c7439a13af5682a223474386f9a1cc6e8..d54ebfbd6fb1abb1d2239800f503c6d478d0313e 100644
--- a/sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027
+++ b/sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027
@@ -1,2 +1,2 @@
 std(x) - Returns the standard deviation of a set of numbers
-Synonyms: stddev_pop, stddev
\ No newline at end of file
+Synonyms: stddev_pop, stddev
diff --git a/sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 b/sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92
index 74c6e1eacc379b4a03029c9ddf5f1425cf130b36..c8b11307792b39dd6dcf17b75bf8c9b714cea3aa 100644
--- a/sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92
+++ b/sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92
@@ -1 +1 @@
-stddev(x) - Returns the standard deviation of a set of numbers
\ No newline at end of file
+stddev(x) - Returns the standard deviation of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d b/sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d
index 1c461b68c144091f5d02000ce4b4dda1b3289d2c..5f674788180e8d0b5dd87d6701bbd64d7a781730 100644
--- a/sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d
+++ b/sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d
@@ -1,2 +1,2 @@
 stddev(x) - Returns the standard deviation of a set of numbers
-Synonyms: stddev_pop, std
\ No newline at end of file
+Synonyms: stddev_pop, std
diff --git a/sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 b/sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12
index 741771707f2803fc625038cdaf702e8603085a66..98b461bd09708bc2f17ec9d9ce7a438a6ee4cb5f 100644
--- a/sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12
+++ b/sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12
@@ -1 +1 @@
-Function 'udf_stddev_pop' does not exist.
\ No newline at end of file
+Function 'udf_stddev_pop' does not exist.
diff --git a/sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 b/sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10
index 741771707f2803fc625038cdaf702e8603085a66..98b461bd09708bc2f17ec9d9ce7a438a6ee4cb5f 100644
--- a/sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10
+++ b/sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10
@@ -1 +1 @@
-Function 'udf_stddev_pop' does not exist.
\ No newline at end of file
+Function 'udf_stddev_pop' does not exist.
diff --git a/sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4
index c1fb091d2c4258dec67830084861ce9f3acbb264..cc3c37142270efd6bccf5f4edaffe05b26fd7351 100644
--- a/sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4
+++ b/sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4
@@ -1 +1 @@
-stddev_samp(x) - Returns the sample standard deviation of a set of numbers
\ No newline at end of file
+stddev_samp(x) - Returns the sample standard deviation of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4
index c1fb091d2c4258dec67830084861ce9f3acbb264..cc3c37142270efd6bccf5f4edaffe05b26fd7351 100644
--- a/sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4
+++ b/sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4
@@ -1 +1 @@
-stddev_samp(x) - Returns the sample standard deviation of a set of numbers
\ No newline at end of file
+stddev_samp(x) - Returns the sample standard deviation of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4
index c1fb091d2c4258dec67830084861ce9f3acbb264..cc3c37142270efd6bccf5f4edaffe05b26fd7351 100644
--- a/sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4
+++ b/sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4
@@ -1 +1 @@
-stddev_samp(x) - Returns the sample standard deviation of a set of numbers
\ No newline at end of file
+stddev_samp(x) - Returns the sample standard deviation of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4
index c1fb091d2c4258dec67830084861ce9f3acbb264..cc3c37142270efd6bccf5f4edaffe05b26fd7351 100644
--- a/sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4
+++ b/sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4
@@ -1 +1 @@
-stddev_samp(x) - Returns the sample standard deviation of a set of numbers
\ No newline at end of file
+stddev_samp(x) - Returns the sample standard deviation of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 b/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3
index c61c4e11354103a1d4ee3b764951cc114a07810b..4568e35cb1a21504a3dbdb605ac6a06434396066 100644
--- a/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3
+++ b/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3
@@ -1 +1 @@
-There is no documentation for function 'string'
\ No newline at end of file
+There is no documentation for function 'string'
diff --git a/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d b/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d
index c61c4e11354103a1d4ee3b764951cc114a07810b..4568e35cb1a21504a3dbdb605ac6a06434396066 100644
--- a/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d
+++ b/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d
@@ -1 +1 @@
-There is no documentation for function 'string'
\ No newline at end of file
+There is no documentation for function 'string'
diff --git a/sql/hive/src/test/resources/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd b/sql/hive/src/test/resources/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd
deleted file mode 100644
index c4efdaebc8aab87030fd821a8d897c9c2ddae9c5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd
+++ /dev/null
@@ -1 +0,0 @@
-substr(str, pos[, len]) - returns the substring of str that starts at pos and is of length len orsubstr(bin, pos[, len]) - returns the slice of byte array that starts at pos and is of length len
diff --git a/sql/hive/src/test/resources/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 b/sql/hive/src/test/resources/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920
deleted file mode 100644
index 44958996269c9ec30b49801386b8095ca43986ea..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920
+++ /dev/null
@@ -1,10 +0,0 @@
-substr(str, pos[, len]) - returns the substring of str that starts at pos and is of length len orsubstr(bin, pos[, len]) - returns the slice of byte array that starts at pos and is of length len
-Synonyms: substring
-pos is a 1-based index. If pos<0 the starting position is determined by counting backwards from the end of str.
-Example:
-   > SELECT substr('Facebook', 5) FROM src LIMIT 1;
-  'book'
-  > SELECT substr('Facebook', -5) FROM src LIMIT 1;
-  'ebook'
-  > SELECT substr('Facebook', 5, 1) FROM src LIMIT 1;
-  'b'
diff --git a/sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 b/sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1
deleted file mode 100644
index 9516b839a395606cd2bdcccabaaee456b2bbbd4f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1
+++ /dev/null
@@ -1 +0,0 @@
-NULL	NULL	NULL	NULL	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 b/sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122
deleted file mode 100644
index 7f6c057b5eebd6ebcd0edcf641b8bc7f59642c96..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122
+++ /dev/null
@@ -1 +0,0 @@
-												
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a b/sql/hive/src/test/resources/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a
deleted file mode 100644
index 042ca15005869fb02857de10504d09f5081c6c21..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a
+++ /dev/null
@@ -1 +0,0 @@
-CDEF	CDEF	CDEFG	CDEFG	ABC	ABC	BC	C	ABC	BC	A	A	A
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 b/sql/hive/src/test/resources/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8
deleted file mode 100644
index 85346dc923b4adb5d164b798e66a364071a2338a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8
+++ /dev/null
@@ -1 +0,0 @@
-A	AB	ABC	ABC	A	AB	ABC	ABC	B	BC	BC	BC	C	C	C	C	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 b/sql/hive/src/test/resources/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6
deleted file mode 100644
index 9d607f4d84214b916b08a2e630695f558f06c60e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6
+++ /dev/null
@@ -1 +0,0 @@
-C	C	C	C	B	BC	BC	BC	A	AB	ABC	ABC	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 b/sql/hive/src/test/resources/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496
deleted file mode 100644
index 042ca15005869fb02857de10504d09f5081c6c21..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496
+++ /dev/null
@@ -1 +0,0 @@
-CDEF	CDEF	CDEFG	CDEFG	ABC	ABC	BC	C	ABC	BC	A	A	A
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade b/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade
deleted file mode 100644
index c14d8d4279c5fb954e426cea240fe7691dfd86a1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade
+++ /dev/null
@@ -1 +0,0 @@
-NULL	NULL	NULL	NULL	NULL	A	AB	ABC	ABC	A	AB	ABC	ABC	B	BC	BC	BC	C	C	C	C		C	C	C	C	B	BC	BC	BC	A	AB	ABC	ABC	
diff --git a/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d b/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d
deleted file mode 100644
index 94a57d86c88fdaef6744dd267b726c60a1a580ef..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d
+++ /dev/null
@@ -1 +0,0 @@
-玩	玩	玩玩玩 abc	玩玩玩
diff --git a/sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece b/sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece
index b0ade52e69a9d2d2376bb197ccdac388c448aa5b..6a2376d85e7bcb6bc874d304f4e7853ba0f2544b 100644
--- a/sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece
+++ b/sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece
@@ -1 +1 @@
-substring(str, pos[, len]) - returns the substring of str that starts at pos and is of length len orsubstring(bin, pos[, len]) - returns the slice of byte array that starts at pos and is of length len
\ No newline at end of file
+substring(str, pos[, len]) - returns the substring of str that starts at pos and is of length len orsubstring(bin, pos[, len]) - returns the slice of byte array that starts at pos and is of length len
diff --git a/sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 b/sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8
index 1d61dc45180871fc2d641ec230db40e037e712ec..32941e727ce383a363b4bf7584b73c9d14ae2e3c 100644
--- a/sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8
+++ b/sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8
@@ -7,4 +7,4 @@ Example:
   > SELECT substring('Facebook', -5) FROM src LIMIT 1;
   'ebook'
   > SELECT substring('Facebook', 5, 1) FROM src LIMIT 1;
-  'b'
\ No newline at end of file
+  'b'
diff --git a/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 b/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3
index 0f4a4dce9dd3d3809a9d5ca451b7e3fbc59b3ab1..1ad8be1242f0010f069fddeaf91527be54d49f3e 100644
--- a/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3
+++ b/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3
@@ -1 +1 @@
-a - b - Returns the difference a-b
\ No newline at end of file
+a - b - Returns the difference a-b
diff --git a/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 b/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722
index 0f4a4dce9dd3d3809a9d5ca451b7e3fbc59b3ab1..1ad8be1242f0010f069fddeaf91527be54d49f3e 100644
--- a/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722
+++ b/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722
@@ -1 +1 @@
-a - b - Returns the difference a-b
\ No newline at end of file
+a - b - Returns the difference a-b
diff --git a/sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 b/sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31
index 744935026110c31d6edb222a44f1ebce1dd9f074..3c8fbd8c4d90098aa38b555a30767588edf3716f 100644
--- a/sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31
+++ b/sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31
@@ -1 +1 @@
-sum(x) - Returns the sum of a set of numbers
\ No newline at end of file
+sum(x) - Returns the sum of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b b/sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b
index 744935026110c31d6edb222a44f1ebce1dd9f074..3c8fbd8c4d90098aa38b555a30767588edf3716f 100644
--- a/sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b
+++ b/sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b
@@ -1 +1 @@
-sum(x) - Returns the sum of a set of numbers
\ No newline at end of file
+sum(x) - Returns the sum of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 b/sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31
index 744935026110c31d6edb222a44f1ebce1dd9f074..3c8fbd8c4d90098aa38b555a30767588edf3716f 100644
--- a/sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31
+++ b/sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31
@@ -1 +1 @@
-sum(x) - Returns the sum of a set of numbers
\ No newline at end of file
+sum(x) - Returns the sum of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b b/sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b
index 744935026110c31d6edb222a44f1ebce1dd9f074..3c8fbd8c4d90098aa38b555a30767588edf3716f 100644
--- a/sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b
+++ b/sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b
@@ -1 +1 @@
-sum(x) - Returns the sum of a set of numbers
\ No newline at end of file
+sum(x) - Returns the sum of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/newline-1-a19a19272149c732977c37e043910505 b/sql/hive/src/test/resources/golden/udf_tan-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/newline-1-a19a19272149c732977c37e043910505
rename to sql/hive/src/test/resources/golden/udf_tan-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 b/sql/hive/src/test/resources/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878
deleted file mode 100644
index c91072e69740b04f06df5df0e3c9446296a90cab..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878
+++ /dev/null
@@ -1 +0,0 @@
-tan(x) - returns the tangent of x (x is in radians)
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/sql/hive/src/test/resources/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505
deleted file mode 100644
index 4cc0d466f00f49df734eb820f27b478909ad0ccb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505
+++ /dev/null
@@ -1,4 +0,0 @@
-tan(x) - returns the tangent of x (x is in radians)
-Example:
-   > SELECT tan(0) FROM src LIMIT 1;
-  1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_tan-1-c21aa640b4edabf6613dd705d029c878 b/sql/hive/src/test/resources/golden/udf_tan-1-c21aa640b4edabf6613dd705d029c878
new file mode 100644
index 0000000000000000000000000000000000000000..3ac5f19b2f2f385448fa2d68f418604413e160dd
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_tan-1-c21aa640b4edabf6613dd705d029c878
@@ -0,0 +1 @@
+tan(x) - returns the tangent of x (x is in radians)
diff --git a/sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 b/sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_tan-2-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/sql/hive/src/test/resources/golden/udf_tan-2-a8c8eaa832aa9a4345b2fb9cd5e1d505
new file mode 100644
index 0000000000000000000000000000000000000000..f26ecfe4a29d3671086194a1821ac3b7256dce8f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_tan-2-a8c8eaa832aa9a4345b2fb9cd5e1d505
@@ -0,0 +1,4 @@
+tan(x) - returns the tangent of x (x is in radians)
+Example:
+   > SELECT tan(0) FROM src LIMIT 1;
+  1
diff --git a/sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 b/sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97
deleted file mode 100644
index 04ec33c11c3b4bcbdb21a41a162b5c70824afe34..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97
+++ /dev/null
@@ -1 +0,0 @@
-1.5574077246549023	-0.29100619138474915	-1.5574077246549023
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9 b/sql/hive/src/test/resources/golden/udf_tan-3-8b46b68ff11c5fb05fb4fd7605895f0b
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9
rename to sql/hive/src/test/resources/golden/udf_tan-3-8b46b68ff11c5fb05fb4fd7605895f0b
diff --git a/sql/hive/src/test/resources/golden/udf_tan-4-769fde617744ccfaa29cefec81b8704c b/sql/hive/src/test/resources/golden/udf_tan-4-769fde617744ccfaa29cefec81b8704c
new file mode 100644
index 0000000000000000000000000000000000000000..e9f2fa411fa3309202e629f9c994d3260ea5b017
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_tan-4-769fde617744ccfaa29cefec81b8704c
@@ -0,0 +1 @@
+1.5574077246549023	-0.29100619138474915	-1.5574077246549023
diff --git a/sql/hive/src/test/resources/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 b/sql/hive/src/test/resources/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878
deleted file mode 100644
index c91072e69740b04f06df5df0e3c9446296a90cab..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878
+++ /dev/null
@@ -1 +0,0 @@
-tan(x) - returns the tangent of x (x is in radians)
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/sql/hive/src/test/resources/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505
deleted file mode 100644
index 4cc0d466f00f49df734eb820f27b478909ad0ccb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505
+++ /dev/null
@@ -1,4 +0,0 @@
-tan(x) - returns the tangent of x (x is in radians)
-Example:
-   > SELECT tan(0) FROM src LIMIT 1;
-  1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_tan-5-c21aa640b4edabf6613dd705d029c878 b/sql/hive/src/test/resources/golden/udf_tan-5-c21aa640b4edabf6613dd705d029c878
new file mode 100644
index 0000000000000000000000000000000000000000..3ac5f19b2f2f385448fa2d68f418604413e160dd
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_tan-5-c21aa640b4edabf6613dd705d029c878
@@ -0,0 +1 @@
+tan(x) - returns the tangent of x (x is in radians)
diff --git a/sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 b/sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85
deleted file mode 100644
index fe3a0735d98b88f091b15ff86c2dc7a167bef217..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85
+++ /dev/null
@@ -1 +0,0 @@
-NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_tan-6-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/sql/hive/src/test/resources/golden/udf_tan-6-a8c8eaa832aa9a4345b2fb9cd5e1d505
new file mode 100644
index 0000000000000000000000000000000000000000..f26ecfe4a29d3671086194a1821ac3b7256dce8f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_tan-6-a8c8eaa832aa9a4345b2fb9cd5e1d505
@@ -0,0 +1,4 @@
+tan(x) - returns the tangent of x (x is in radians)
+Example:
+   > SELECT tan(0) FROM src LIMIT 1;
+  1
diff --git a/sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 b/sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97
deleted file mode 100644
index 04ec33c11c3b4bcbdb21a41a162b5c70824afe34..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97
+++ /dev/null
@@ -1 +0,0 @@
-1.5574077246549023	-0.29100619138474915	-1.5574077246549023
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 b/sql/hive/src/test/resources/golden/udf_tan-7-8b46b68ff11c5fb05fb4fd7605895f0b
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4
rename to sql/hive/src/test/resources/golden/udf_tan-7-8b46b68ff11c5fb05fb4fd7605895f0b
diff --git a/sql/hive/src/test/resources/golden/udf_tan-8-769fde617744ccfaa29cefec81b8704c b/sql/hive/src/test/resources/golden/udf_tan-8-769fde617744ccfaa29cefec81b8704c
new file mode 100644
index 0000000000000000000000000000000000000000..e9f2fa411fa3309202e629f9c994d3260ea5b017
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_tan-8-769fde617744ccfaa29cefec81b8704c
@@ -0,0 +1 @@
+1.5574077246549023	-0.29100619138474915	-1.5574077246549023
diff --git a/sql/hive/src/test/resources/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad b/sql/hive/src/test/resources/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad
deleted file mode 100644
index 89617d8af00851d3eb7257dcd505993670167dfb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad
+++ /dev/null
@@ -1,10 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_CREATEFUNCTION testlength 'org.apache.hadoop.hive.ql.udf.UDFTestLength')
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-
-
diff --git a/sql/hive/src/test/resources/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 b/sql/hive/src/test/resources/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55
deleted file mode 100644
index 6d967660b3d41e2d0b9c85a73a6a6472f6d159d6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55
+++ /dev/null
@@ -1,10 +0,0 @@
-ABSTRACT SYNTAX TREE:
-  (TOK_CREATEFUNCTION testlength2 'org.apache.hadoop.hive.ql.udf.UDFTestLength2')
-
-STAGE DEPENDENCIES:
-  Stage-0 is a root stage
-
-STAGE PLANS:
-  Stage: Stage-0
-
-
diff --git a/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d b/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d
index 1b13d42908539f9a891a7e0676bcc13169770345..898364b6d3e0b02252415722ce1108fda9ee00e4 100644
--- a/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d
+++ b/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d
@@ -1 +1 @@
-There is no documentation for function 'tinyint'
\ No newline at end of file
+There is no documentation for function 'tinyint'
diff --git a/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 b/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426
index 1b13d42908539f9a891a7e0676bcc13169770345..898364b6d3e0b02252415722ce1108fda9ee00e4 100644
--- a/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426
+++ b/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426
@@ -1 +1 @@
-There is no documentation for function 'tinyint'
\ No newline at end of file
+There is no documentation for function 'tinyint'
diff --git a/sql/hive/src/test/resources/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 b/sql/hive/src/test/resources/golden/udf_to_byte-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8
rename to sql/hive/src/test/resources/golden/udf_to_byte-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 b/sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3
deleted file mode 100644
index d00491fd7e5bb6fa28c517a0bb32b8b506539d4d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3
+++ /dev/null
@@ -1 +0,0 @@
-1
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 b/sql/hive/src/test/resources/golden/udf_to_byte-1-94eb069fb446b7758f7e06386486bec9
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7
rename to sql/hive/src/test/resources/golden/udf_to_byte-1-94eb069fb446b7758f7e06386486bec9
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 b/sql/hive/src/test/resources/golden/udf_to_byte-2-233102b562824cf38010868478e91e1
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9
rename to sql/hive/src/test/resources/golden/udf_to_byte-2-233102b562824cf38010868478e91e1
diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc b/sql/hive/src/test/resources/golden/udf_to_byte-3-5dc0e4c21764683d98700860d2c8ab31
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc
rename to sql/hive/src/test/resources/golden/udf_to_byte-3-5dc0e4c21764683d98700860d2c8ab31
diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 b/sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579
deleted file mode 100644
index 3a2e3f4984a0ee55900f8c7894844c563d2c2744..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579
+++ /dev/null
@@ -1 +0,0 @@
--1
diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b b/sql/hive/src/test/resources/golden/udf_to_byte-4-dafb27507b4d30fd2231680f9ea80c82
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b
rename to sql/hive/src/test/resources/golden/udf_to_byte-4-dafb27507b4d30fd2231680f9ea80c82
diff --git a/sql/hive/src/test/resources/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/udf_to_byte-5-eb6600cd2260e8e75253e7844c0d7dc2
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22
rename to sql/hive/src/test/resources/golden/udf_to_byte-5-eb6600cd2260e8e75253e7844c0d7dc2
diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 b/sql/hive/src/test/resources/golden/udf_to_byte-6-489cd2d26b9efde2cdbff19254289371
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67
rename to sql/hive/src/test/resources/golden/udf_to_byte-6-489cd2d26b9efde2cdbff19254289371
diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 b/sql/hive/src/test/resources/golden/udf_to_byte-7-1eaba393f93af1763dd761172fb78d52
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404
rename to sql/hive/src/test/resources/golden/udf_to_byte-7-1eaba393f93af1763dd761172fb78d52
diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 b/sql/hive/src/test/resources/golden/udf_to_byte-8-8fe36cf8fba87514744a89fe50414f79
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718
rename to sql/hive/src/test/resources/golden/udf_to_byte-8-8fe36cf8fba87514744a89fe50414f79
diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 b/sql/hive/src/test/resources/golden/udf_to_byte-9-322163c32973ccc3a5168463db7a8589
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25
rename to sql/hive/src/test/resources/golden/udf_to_byte-9-322163c32973ccc3a5168463db7a8589
diff --git a/sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 b/sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166
index 54a99d2a94ef4db4964f7cff9dbaf4fcf97a4042..caf10b953f895ae05896894401941e68f1f08198 100644
--- a/sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166
+++ b/sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166
@@ -1 +1 @@
-to_date(expr) - Extracts the date part of the date or datetime expression expr
\ No newline at end of file
+to_date(expr) - Extracts the date part of the date or datetime expression expr
diff --git a/sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e b/sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e
index 81ddc59fcb531601a1a729d67bc31746cee7bbbb..d9cc4956e5f8408c869001817877181cf5bada50 100644
--- a/sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e
+++ b/sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e
@@ -1,4 +1,4 @@
 to_date(expr) - Extracts the date part of the date or datetime expression expr
 Example:
-   > SELECT to_date('2009-30-07 04:17:52') FROM src LIMIT 1;
-  '2009-30-07'
\ No newline at end of file
+   > SELECT to_date('2009-07-30 04:17:52') FROM src LIMIT 1;
+  '2009-07-30'
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe b/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe
deleted file mode 100644
index 319fde05380bcbb83e139e4673641c6ad98cf5fd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe
+++ /dev/null
@@ -1 +0,0 @@
--7.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f b/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f
deleted file mode 100644
index 8c1c4fe62b6c2fed3f5dd23d426bef08c14d3837..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f
+++ /dev/null
@@ -1 +0,0 @@
--18.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 b/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68
deleted file mode 100644
index 1b650de78904f9c04f1ef6d35f3ec35e69ed10b4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68
+++ /dev/null
@@ -1 +0,0 @@
--129.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 b/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8
deleted file mode 100644
index 3a3bd0df03b5b1b1144a6eb3abec58a646c84c4c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8
+++ /dev/null
@@ -1 +0,0 @@
--1025.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 b/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6
deleted file mode 100644
index 38f7ad5afa0abcadd03eb86758c8dcfd4b6aa386..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6
+++ /dev/null
@@ -1 +0,0 @@
--3.140000104904175
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 b/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5
deleted file mode 100644
index 01e913dbfe725d2fdb0b96b99af47aed063f1792..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5
+++ /dev/null
@@ -1 +0,0 @@
--3.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 b/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982
deleted file mode 100644
index f45d1f04dc9209a2f496a4d1defd4264ae3125fa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982
+++ /dev/null
@@ -1 +0,0 @@
--38.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 b/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971
deleted file mode 100644
index 319fde05380bcbb83e139e4673641c6ad98cf5fd..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971
+++ /dev/null
@@ -1 +0,0 @@
--7.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce b/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce
deleted file mode 100644
index 8c1c4fe62b6c2fed3f5dd23d426bef08c14d3837..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce
+++ /dev/null
@@ -1 +0,0 @@
--18.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 b/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507
deleted file mode 100644
index 1b650de78904f9c04f1ef6d35f3ec35e69ed10b4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507
+++ /dev/null
@@ -1 +0,0 @@
--129.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 b/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201
deleted file mode 100644
index 3a3bd0df03b5b1b1144a6eb3abec58a646c84c4c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201
+++ /dev/null
@@ -1 +0,0 @@
--1025.0
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 b/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002
deleted file mode 100644
index 01e913dbfe725d2fdb0b96b99af47aed063f1792..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002
+++ /dev/null
@@ -1 +0,0 @@
--3.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 b/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1
deleted file mode 100644
index 01e913dbfe725d2fdb0b96b99af47aed063f1792..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1
+++ /dev/null
@@ -1 +0,0 @@
--3.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c b/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c
deleted file mode 100644
index f45d1f04dc9209a2f496a4d1defd4264ae3125fa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c
+++ /dev/null
@@ -1 +0,0 @@
--38.14
diff --git a/sql/hive/src/test/resources/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb b/sql/hive/src/test/resources/golden/udf_to_long-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb
rename to sql/hive/src/test/resources/golden/udf_to_long-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 b/sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0
deleted file mode 100644
index d00491fd7e5bb6fa28c517a0bb32b8b506539d4d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0
+++ /dev/null
@@ -1 +0,0 @@
-1
diff --git a/sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 b/sql/hive/src/test/resources/golden/udf_to_long-1-8aeb3ba62d4a0cecfff363741e8042f6
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89
rename to sql/hive/src/test/resources/golden/udf_to_long-1-8aeb3ba62d4a0cecfff363741e8042f6
diff --git a/sql/hive/src/test/resources/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 b/sql/hive/src/test/resources/golden/udf_to_long-2-7e8fa1ae8d00a121ec14941a48d24947
similarity index 100%
rename from sql/hive/src/test/resources/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619
rename to sql/hive/src/test/resources/golden/udf_to_long-2-7e8fa1ae8d00a121ec14941a48d24947
diff --git a/sql/hive/src/test/resources/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 b/sql/hive/src/test/resources/golden/udf_to_long-3-6e5936fba8e7486beb9ab998548bbe9b
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5
rename to sql/hive/src/test/resources/golden/udf_to_long-3-6e5936fba8e7486beb9ab998548bbe9b
diff --git a/sql/hive/src/test/resources/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 b/sql/hive/src/test/resources/golden/udf_to_long-4-8c284b082a256abf0426d4f6f1971703
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613
rename to sql/hive/src/test/resources/golden/udf_to_long-4-8c284b082a256abf0426d4f6f1971703
diff --git a/sql/hive/src/test/resources/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 b/sql/hive/src/test/resources/golden/udf_to_long-5-6bb29b93f6b0f1427ba93efb4e78810a
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6
rename to sql/hive/src/test/resources/golden/udf_to_long-5-6bb29b93f6b0f1427ba93efb4e78810a
diff --git a/sql/hive/src/test/resources/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d b/sql/hive/src/test/resources/golden/udf_to_long-6-290b5a4ce01563482e81b3b532ebf9db
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d
rename to sql/hive/src/test/resources/golden/udf_to_long-6-290b5a4ce01563482e81b3b532ebf9db
diff --git a/sql/hive/src/test/resources/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 b/sql/hive/src/test/resources/golden/udf_to_long-7-da20f84586dac3e50ee9d5b9078f44db
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0
rename to sql/hive/src/test/resources/golden/udf_to_long-7-da20f84586dac3e50ee9d5b9078f44db
diff --git a/sql/hive/src/test/resources/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e b/sql/hive/src/test/resources/golden/udf_to_long-8-90f068b4b6275bdd1c4c431fb7fa90e2
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e
rename to sql/hive/src/test/resources/golden/udf_to_long-8-90f068b4b6275bdd1c4c431fb7fa90e2
diff --git a/sql/hive/src/test/resources/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 b/sql/hive/src/test/resources/golden/udf_to_long-9-cc8b79539085fe0e00f672b562c51cd0
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9
rename to sql/hive/src/test/resources/golden/udf_to_long-9-cc8b79539085fe0e00f672b562c51cd0
diff --git a/sql/hive/src/test/resources/golden/orc_create-32-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_to_short-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/orc_create-32-16367c381d4b189b3640c92511244bfe
rename to sql/hive/src/test/resources/golden/udf_to_short-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a b/sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a
deleted file mode 100644
index 7951defec192aa41c72f62ac9c9f4b001cdaaba8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a
+++ /dev/null
@@ -1 +0,0 @@
-NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b b/sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b
deleted file mode 100644
index d00491fd7e5bb6fa28c517a0bb32b8b506539d4d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b
+++ /dev/null
@@ -1 +0,0 @@
-1
diff --git a/sql/hive/src/test/resources/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a b/sql/hive/src/test/resources/golden/udf_to_short-1-94f15fe043839493107058a06a210cf7
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a
rename to sql/hive/src/test/resources/golden/udf_to_short-1-94f15fe043839493107058a06a210cf7
diff --git a/sql/hive/src/test/resources/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 b/sql/hive/src/test/resources/golden/udf_to_short-2-981e4cb6654fde7eb4634c7ad72f8570
similarity index 100%
rename from sql/hive/src/test/resources/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517
rename to sql/hive/src/test/resources/golden/udf_to_short-2-981e4cb6654fde7eb4634c7ad72f8570
diff --git a/sql/hive/src/test/resources/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 b/sql/hive/src/test/resources/golden/udf_to_short-3-b259ee30ecf279bb4ad12d1515ca2767
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8
rename to sql/hive/src/test/resources/golden/udf_to_short-3-b259ee30ecf279bb4ad12d1515ca2767
diff --git a/sql/hive/src/test/resources/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce b/sql/hive/src/test/resources/golden/udf_to_short-4-40ffb132d5641645e2b8043dc056fb0
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce
rename to sql/hive/src/test/resources/golden/udf_to_short-4-40ffb132d5641645e2b8043dc056fb0
diff --git a/sql/hive/src/test/resources/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 b/sql/hive/src/test/resources/golden/udf_to_short-5-5d6c46b0154d1073c035a79dbf612479
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8
rename to sql/hive/src/test/resources/golden/udf_to_short-5-5d6c46b0154d1073c035a79dbf612479
diff --git a/sql/hive/src/test/resources/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 b/sql/hive/src/test/resources/golden/udf_to_short-6-6561b41835a21f973cbbc2dd80eef87f
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5
rename to sql/hive/src/test/resources/golden/udf_to_short-6-6561b41835a21f973cbbc2dd80eef87f
diff --git a/sql/hive/src/test/resources/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 b/sql/hive/src/test/resources/golden/udf_to_short-7-9f83813005b639a23901ca6ff87ff473
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397
rename to sql/hive/src/test/resources/golden/udf_to_short-7-9f83813005b639a23901ca6ff87ff473
diff --git a/sql/hive/src/test/resources/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d b/sql/hive/src/test/resources/golden/udf_to_short-8-885656e165feb3a674cf636dbf08716c
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d
rename to sql/hive/src/test/resources/golden/udf_to_short-8-885656e165feb3a674cf636dbf08716c
diff --git a/sql/hive/src/test/resources/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 b/sql/hive/src/test/resources/golden/udf_to_short-9-750382fa1a1b3ed5dca0d549d3a68996
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4
rename to sql/hive/src/test/resources/golden/udf_to_short-9-750382fa1a1b3ed5dca0d549d3a68996
diff --git a/sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc b/sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc
deleted file mode 100644
index 7951defec192aa41c72f62ac9c9f4b001cdaaba8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc
+++ /dev/null
@@ -1 +0,0 @@
-NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 b/sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44
deleted file mode 100644
index ef2f5130b8575b3ad3c998555dc34d8d993cdf06..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44
+++ /dev/null
@@ -1 +0,0 @@
-TRUE
diff --git a/sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 b/sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5
deleted file mode 100644
index d00491fd7e5bb6fa28c517a0bb32b8b506539d4d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5
+++ /dev/null
@@ -1 +0,0 @@
-1
diff --git a/sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 b/sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15
deleted file mode 100644
index 6cc8a61f8f6adf9bc335e87cec0dba34e4c72027..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15
+++ /dev/null
@@ -1 +0,0 @@
--18
diff --git a/sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 b/sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21
deleted file mode 100644
index 9828ff22b667b91733f9a7c514f51e8c3f2caf5b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21
+++ /dev/null
@@ -1 +0,0 @@
--129
diff --git a/sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303 b/sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303
deleted file mode 100644
index 450a6125550e5010e12faf815cca5e6a1d3a06c7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303
+++ /dev/null
@@ -1 +0,0 @@
--1025
diff --git a/sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 b/sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384
deleted file mode 100644
index 01e913dbfe725d2fdb0b96b99af47aed063f1792..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384
+++ /dev/null
@@ -1 +0,0 @@
--3.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 b/sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15
deleted file mode 100644
index 01e913dbfe725d2fdb0b96b99af47aed063f1792..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15
+++ /dev/null
@@ -1 +0,0 @@
--3.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea b/sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea
deleted file mode 100644
index 01e913dbfe725d2fdb0b96b99af47aed063f1792..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea
+++ /dev/null
@@ -1 +0,0 @@
--3.14
diff --git a/sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b b/sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b
deleted file mode 100644
index bc56c4d89448a963d0b6486989809d653dab9a97..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b
+++ /dev/null
@@ -1 +0,0 @@
-Foo
diff --git a/sql/hive/src/test/resources/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 b/sql/hive/src/test/resources/golden/udf_translate-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1
rename to sql/hive/src/test/resources/golden/udf_translate-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 b/sql/hive/src/test/resources/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241
deleted file mode 100644
index 4255dc76f501e81d536fe1d848b951b13b1c5c12..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241
+++ /dev/null
@@ -1 +0,0 @@
-translate(input, from, to) - translates the input string by replacing the characters present in the from string with the corresponding characters in the to string
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_translate-1-7fe940a2b26fa19a3cfee39e56fb1241 b/sql/hive/src/test/resources/golden/udf_translate-1-7fe940a2b26fa19a3cfee39e56fb1241
new file mode 100644
index 0000000000000000000000000000000000000000..e92c71fd3ccf95d94efd012362565b5e0fe93b94
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_translate-1-7fe940a2b26fa19a3cfee39e56fb1241
@@ -0,0 +1 @@
+translate(input, from, to) - translates the input string by replacing the characters present in the from string with the corresponding characters in the to string
diff --git a/sql/hive/src/test/resources/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 b/sql/hive/src/test/resources/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8
deleted file mode 100644
index e7beead53b3995ec3e2851e9dbadd475d5f0352e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8
+++ /dev/null
@@ -1,13 +0,0 @@
-translate(input, from, to) - translates the input string by replacing the characters present in the from string with the corresponding characters in the to string
-translate(string input, string from, string to) is an equivalent function to translate in PostGreSQL. It works on a character by character basis on the input string (first parameter). A character in the input is checked for presence in the from string (second parameter). If a match happens, the character from to string (third parameter) which appears at the same index as the character in from string is obtained. This character is emitted in the output string  instead of the original character from the input string. If the to string is shorter than the from string, there may not be a character present at the same index in the to string. In such a case, nothing is emitted for the original character and it's deleted from the output string.
-For example,
-
-translate('abcdef', 'adc', '19') returns '1b9ef' replacing 'a' with '1', 'd' with '9' and removing 'c' from the input string
-
-translate('a b c d', ' ', '') return 'abcd' removing all spaces from the input string
-
-If the same character is present multiple times in the input string, the first occurence of the character is the one that's considered for matching. However, it is not recommended to have the same character more than once in the from string since it's not required and adds to confusion.
-
-For example,
-
-translate('abcdef', 'ada', '192') returns '1bc9ef' replaces 'a' with '1' and 'd' with '9' ignoring the second occurence of 'a' in the from string mapping it to '2'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 b/sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4
deleted file mode 100644
index 7d44692e4f7c058c73622e612745f38ad4e81981..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4
+++ /dev/null
@@ -1 +0,0 @@
-12cd	12cd
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_translate-10-ca7c17e78c6a3d4e19dbd66622a87eae b/sql/hive/src/test/resources/golden/udf_translate-10-ca7c17e78c6a3d4e19dbd66622a87eae
new file mode 100644
index 0000000000000000000000000000000000000000..ae8343d33bc11593ff8fa04792977bcd6680df05
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_translate-10-ca7c17e78c6a3d4e19dbd66622a87eae
@@ -0,0 +1 @@
+NULL	NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/udf_translate-11-40c4e7adff4dde739d7797d212892c5a b/sql/hive/src/test/resources/golden/udf_translate-11-40c4e7adff4dde739d7797d212892c5a
new file mode 100644
index 0000000000000000000000000000000000000000..2e2f35d06c0424da38b7852ddf5e53bff96b6e9f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_translate-11-40c4e7adff4dde739d7797d212892c5a
@@ -0,0 +1 @@
+12cd	12cd
diff --git a/sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b b/sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b
deleted file mode 100644
index 90e0d872f8bfc87ea322777f5c5fb695771b6648..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b
+++ /dev/null
@@ -1 +0,0 @@
-123d
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 b/sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05
deleted file mode 100644
index 0770d02cb23033b54bae3e6da973b6566786369a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05
+++ /dev/null
@@ -1 +0,0 @@
-Ãbcd
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_translate-12-d81fd3267ec96cff31079142bf5d49bf b/sql/hive/src/test/resources/golden/udf_translate-12-d81fd3267ec96cff31079142bf5d49bf
new file mode 100644
index 0000000000000000000000000000000000000000..36284d8236d0e43d39459ab0017cdbb3d4c80c6e
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_translate-12-d81fd3267ec96cff31079142bf5d49bf
@@ -0,0 +1 @@
+123d
diff --git a/sql/hive/src/test/resources/golden/udf_translate-13-26085a3eba1a1b34684ec4e6c1723527 b/sql/hive/src/test/resources/golden/udf_translate-13-26085a3eba1a1b34684ec4e6c1723527
new file mode 100644
index 0000000000000000000000000000000000000000..0208f1f4980c2a79347fbaaca141318fbff96675
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_translate-13-26085a3eba1a1b34684ec4e6c1723527
@@ -0,0 +1 @@
+Ãbcd
diff --git a/sql/hive/src/test/resources/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 b/sql/hive/src/test/resources/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8 b/sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8
new file mode 100644
index 0000000000000000000000000000000000000000..9ced4ee32cf0b6d26d03ff0674eb08e8d0a564f0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8
@@ -0,0 +1,13 @@
+translate(input, from, to) - translates the input string by replacing the characters present in the from string with the corresponding characters in the to string
+translate(string input, string from, string to) is an equivalent function to translate in PostGreSQL. It works on a character by character basis on the input string (first parameter). A character in the input is checked for presence in the from string (second parameter). If a match happens, the character from to string (third parameter) which appears at the same index as the character in from string is obtained. This character is emitted in the output string  instead of the original character from the input string. If the to string is shorter than the from string, there may not be a character present at the same index in the to string. In such a case, nothing is emitted for the original character and it's deleted from the output string.
+For example,
+
+translate('abcdef', 'adc', '19') returns '1b9ef' replacing 'a' with '1', 'd' with '9' and removing 'c' from the input string
+
+translate('a b c d', ' ', '') return 'abcd' removing all spaces from the input string
+
+If the same character is present multiple times in the input string, the first occurence of the character is the one that's considered for matching. However, it is not recommended to have the same character more than once in the from string since it's not required and adds to confusion.
+
+For example,
+
+translate('abcdef', 'ada', '192') returns '1bc9ef' replaces 'a' with '1' and 'd' with '9' ignoring the second occurence of 'a' in the from string mapping it to '2'
diff --git a/sql/hive/src/test/resources/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a b/sql/hive/src/test/resources/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 b/sql/hive/src/test/resources/golden/udf_translate-3-42aba80bf1913dd7c64545831f476c58
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279
rename to sql/hive/src/test/resources/golden/udf_translate-3-42aba80bf1913dd7c64545831f476c58
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa b/sql/hive/src/test/resources/golden/udf_translate-4-20904c8be8fed5cbd2d66ead6248a60a
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa
rename to sql/hive/src/test/resources/golden/udf_translate-4-20904c8be8fed5cbd2d66ead6248a60a
diff --git a/sql/hive/src/test/resources/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba b/sql/hive/src/test/resources/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 b/sql/hive/src/test/resources/golden/udf_translate-5-5d4abaf86254bacaa545c769bd7e50ba
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2
rename to sql/hive/src/test/resources/golden/udf_translate-5-5d4abaf86254bacaa545c769bd7e50ba
diff --git a/sql/hive/src/test/resources/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 b/sql/hive/src/test/resources/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b b/sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b
deleted file mode 100644
index 0a54ee40ecb13eac1c69b9047792663f9a39a034..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b
+++ /dev/null
@@ -1 +0,0 @@
-12cd	12d
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad b/sql/hive/src/test/resources/golden/udf_translate-6-f2637240d227f9732d3db76f2e9d3a59
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad
rename to sql/hive/src/test/resources/golden/udf_translate-6-f2637240d227f9732d3db76f2e9d3a59
diff --git a/sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 b/sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47
deleted file mode 100644
index 0a54ee40ecb13eac1c69b9047792663f9a39a034..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47
+++ /dev/null
@@ -1 +0,0 @@
-12cd	12d
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_translate-7-f8de3ab54db5d6a44fddb542b3d99704 b/sql/hive/src/test/resources/golden/udf_translate-7-f8de3ab54db5d6a44fddb542b3d99704
new file mode 100644
index 0000000000000000000000000000000000000000..fbf71ff5ad9eb7cae830de458df9e7932765fd67
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_translate-7-f8de3ab54db5d6a44fddb542b3d99704
@@ -0,0 +1 @@
+12cd	12d
diff --git a/sql/hive/src/test/resources/golden/udf_translate-8-1747ed8fbb4ef889df3db937ee51e2b0 b/sql/hive/src/test/resources/golden/udf_translate-8-1747ed8fbb4ef889df3db937ee51e2b0
new file mode 100644
index 0000000000000000000000000000000000000000..fbf71ff5ad9eb7cae830de458df9e7932765fd67
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_translate-8-1747ed8fbb4ef889df3db937ee51e2b0
@@ -0,0 +1 @@
+12cd	12d
diff --git a/sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 b/sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11
deleted file mode 100644
index 50c123df9d1d36097035dddf6009c49cc3824f52..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11
+++ /dev/null
@@ -1 +0,0 @@
-1bc
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 b/sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1
deleted file mode 100644
index b98d7e1c3402403c1102e807300ad3e217288afa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1
+++ /dev/null
@@ -1 +0,0 @@
-NULL	NULL	NULL	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_translate-9-f1bd0dd5226ee632db3c72c5fc2aaeb0 b/sql/hive/src/test/resources/golden/udf_translate-9-f1bd0dd5226ee632db3c72c5fc2aaeb0
new file mode 100644
index 0000000000000000000000000000000000000000..bb456baa0a6084185922c20aedfdce5a1c49973f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_translate-9-f1bd0dd5226ee632db3c72c5fc2aaeb0
@@ -0,0 +1 @@
+1bc
diff --git a/sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f b/sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f
index f0620a190ae4e14c7b0f6d504d6554de5c688c3b..598aeb205875b09266d215a6ec3b2fd68a805139 100644
--- a/sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f
+++ b/sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f
@@ -1 +1 @@
-trim(str) - Removes the leading and trailing space characters from str 
\ No newline at end of file
+trim(str) - Removes the leading and trailing space characters from str 
diff --git a/sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 b/sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7
index 49a13dd64b286f3b2b6abf6f2eecd0484438042b..b3ec336a877ee94f0f5d23f4687e2be5b030f3ea 100644
--- a/sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7
+++ b/sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7
@@ -1,4 +1,4 @@
 trim(str) - Removes the leading and trailing space characters from str 
 Example:
   > SELECT trim('   facebook  ') FROM src LIMIT 1;
-  'facebook'
\ No newline at end of file
+  'facebook'
diff --git a/sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d b/sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d
index 057c231dccf0e06293fbc1771b92666fe757f225..66412c3ba77a42b156daebcf50e3fcc637029d8f 100644
--- a/sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d
+++ b/sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d
@@ -1 +1 @@
-ucase(str) - Returns str with all characters changed to uppercase
\ No newline at end of file
+ucase(str) - Returns str with all characters changed to uppercase
diff --git a/sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f b/sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f
index 128d1e21fae838504470515b74d9ee24a0c4df32..814e4f0d39dcfd0a90495822e15f33e13c923906 100644
--- a/sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f
+++ b/sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f
@@ -2,4 +2,4 @@ ucase(str) - Returns str with all characters changed to uppercase
 Synonyms: upper
 Example:
   > SELECT ucase('Facebook') FROM src LIMIT 1;
-  'FACEBOOK'
\ No newline at end of file
+  'FACEBOOK'
diff --git a/sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 b/sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47
deleted file mode 100644
index 44b2a42cc26c53977d88671c93c4f5fdc7ab042c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47
+++ /dev/null
@@ -1 +0,0 @@
-unhex(str) - Converts hexadecimal argument to binary
diff --git a/sql/hive/src/test/resources/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 b/sql/hive/src/test/resources/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4
deleted file mode 100644
index 97af3b812a4295d6f54a89a3f80478a2b8f417eb..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4
+++ /dev/null
@@ -1,14 +0,0 @@
-unhex(str) - Converts hexadecimal argument to binary
-Performs the inverse operation of HEX(str). That is, it interprets
-each pair of hexadecimal digits in the argument as a number and
-converts it to the byte representation of the number. The
-resulting characters are returned as a binary string.
-
-Example:
-> SELECT DECODE(UNHEX('4D7953514C'), 'UTF-8') from src limit 1;
-'MySQL'
-
-The characters in the argument string must be legal hexadecimal
-digits: '0' .. '9', 'A' .. 'F', 'a' .. 'f'. If UNHEX() encounters
-any nonhexadecimal digits in the argument, it returns NULL. Also,
-if there are an odd number of characters a leading 0 is appended.
diff --git a/sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 b/sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68
deleted file mode 100644
index b4a6f2b692227cd18a3b47e6d599b29e96cc45a0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68
+++ /dev/null
@@ -1 +0,0 @@
-MySQL	1267	a	-4	
diff --git a/sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 b/sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6
deleted file mode 100644
index 3a67adaf0a9a8c4c5d2daee3e579a369009dcc8c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6
+++ /dev/null
@@ -1 +0,0 @@
-NULL	NULL	NULL
diff --git a/sql/hive/src/test/resources/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c b/sql/hive/src/test/resources/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c
deleted file mode 100644
index 3f3bb2b7ce02e31c7165978a77a7082eea9e3246..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c
+++ /dev/null
@@ -1 +0,0 @@
-create_union(tag, obj1, obj2, obj3, ...) - Creates a union with the object for given tag
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_union-1-f6f01250718087029165e23badc02cd6 b/sql/hive/src/test/resources/golden/udf_union-1-f6f01250718087029165e23badc02cd6
deleted file mode 100644
index 748ed970554013e763e8f78505762433a80ff6e0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_union-1-f6f01250718087029165e23badc02cd6
+++ /dev/null
@@ -1,4 +0,0 @@
-create_union(tag, obj1, obj2, obj3, ...) - Creates a union with the object for given tag
-Example:
-  > SELECT create_union(1, 1, "one") FROM src LIMIT 1;
-  one
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 b/sql/hive/src/test/resources/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_union-3-705d165fec6761744dd19b142c566d61 b/sql/hive/src/test/resources/golden/udf_union-3-705d165fec6761744dd19b142c566d61
deleted file mode 100644
index 4b9ae5402aa538273b97967d4ad61895fc7b803c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_union-3-705d165fec6761744dd19b142c566d61
+++ /dev/null
@@ -1,2 +0,0 @@
-{0:238}	{1:"val_238"}	{1:{"col1":2,"col2":"b"}}
-{0:86}	{0:2.0}	{1:{"col1":2,"col2":"b"}}
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-10-767848077fc2929ccedfd28e50564b19 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/parallel_orderby-10-767848077fc2929ccedfd28e50564b19
rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb b/sql/hive/src/test/resources/golden/udf_unix_timestamp-1-d555c8cd733572bfa8cd3362da9480cb
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb
rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-1-d555c8cd733572bfa8cd3362da9480cb
diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-2-28c40e51e55bed62693e626efda5d9c5 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-2-28c40e51e55bed62693e626efda5d9c5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-2-8a9dbadae706047715cf5f903ff4a724
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724
rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-2-8a9dbadae706047715cf5f903ff4a724
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-3-28c40e51e55bed62693e626efda5d9c5
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434
rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-3-28c40e51e55bed62693e626efda5d9c5
diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-3-732b21d386f2002b87eaf02d0b9951ed b/sql/hive/src/test/resources/golden/udf_unix_timestamp-3-732b21d386f2002b87eaf02d0b9951ed
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-4-6059ff48788d0fb8317fd331172ecea9
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065
rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-4-6059ff48788d0fb8317fd331172ecea9
diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-5-b2e42ebb75cecf09961d36587797f6d0
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0
rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-5-b2e42ebb75cecf09961d36587797f6d0
diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-6-31243f5cb64356425b9f95ba011ac9d6
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6
rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-6-31243f5cb64356425b9f95ba011ac9d6
diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-7-9b0f20bde1aaf9102b67a5498b167f31
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31
rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-7-9b0f20bde1aaf9102b67a5498b167f31
diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d b/sql/hive/src/test/resources/golden/udf_unix_timestamp-8-47f433ff6ccce4c666440cc1a228a96d
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d
rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-8-47f433ff6ccce4c666440cc1a228a96d
diff --git a/sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead b/sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead
index 2e239cca5b860bb5c69606451b91f6aef01b768c..40014886d9646ab98d633d9643fd2d7ac1a71b07 100644
--- a/sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead
+++ b/sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead
@@ -1 +1 @@
-upper(str) - Returns str with all characters changed to uppercase
\ No newline at end of file
+upper(str) - Returns str with all characters changed to uppercase
diff --git a/sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d b/sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d
index 673307e284b2e2f47fdbc7d2dada1053d1453aa6..0fb1bf1823d846ecdb81f04202a149805a9f975b 100644
--- a/sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d
+++ b/sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d
@@ -2,4 +2,4 @@ upper(str) - Returns str with all characters changed to uppercase
 Synonyms: ucase
 Example:
   > SELECT upper('Facebook') FROM src LIMIT 1;
-  'FACEBOOK'
\ No newline at end of file
+  'FACEBOOK'
diff --git a/sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de b/sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de
index 9dbf241e281074f3ef223b5e9e80a4828df179ed..79e293fd4ed52fd5edf276dc1b6ca0e335ab238a 100644
--- a/sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de
+++ b/sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de
@@ -1 +1 @@
-var_pop(x) - Returns the variance of a set of numbers
\ No newline at end of file
+var_pop(x) - Returns the variance of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 b/sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766
index 234a65bd8255d2219f186bf6d08fa43a98becdfb..4b4de5db812e9c493c40795bc4df6aac00eb3cd3 100644
--- a/sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766
+++ b/sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766
@@ -1,2 +1,2 @@
 var_pop(x) - Returns the variance of a set of numbers
-Synonyms: variance
\ No newline at end of file
+Synonyms: variance
diff --git a/sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 b/sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229
index 673deeb031b44574caac15396db41dba347ebfe5..42a442f7a2dedb04ffaa6831d337264de42a8e06 100644
--- a/sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229
+++ b/sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229
@@ -1 +1 @@
-var_samp(x) - Returns the sample variance of a set of numbers
\ No newline at end of file
+var_samp(x) - Returns the sample variance of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b b/sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b
index 673deeb031b44574caac15396db41dba347ebfe5..42a442f7a2dedb04ffaa6831d337264de42a8e06 100644
--- a/sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b
+++ b/sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b
@@ -1 +1 @@
-var_samp(x) - Returns the sample variance of a set of numbers
\ No newline at end of file
+var_samp(x) - Returns the sample variance of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 b/sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229
index 673deeb031b44574caac15396db41dba347ebfe5..42a442f7a2dedb04ffaa6831d337264de42a8e06 100644
--- a/sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229
+++ b/sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229
@@ -1 +1 @@
-var_samp(x) - Returns the sample variance of a set of numbers
\ No newline at end of file
+var_samp(x) - Returns the sample variance of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b b/sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b
index 673deeb031b44574caac15396db41dba347ebfe5..42a442f7a2dedb04ffaa6831d337264de42a8e06 100644
--- a/sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b
+++ b/sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b
@@ -1 +1 @@
-var_samp(x) - Returns the sample variance of a set of numbers
\ No newline at end of file
+var_samp(x) - Returns the sample variance of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 b/sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87
index f58ae0769d5a9dc9a4350ad1df972b5bce6018fb..260300b7d3bd99b8eca4d3aadb512212e1e28756 100644
--- a/sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87
+++ b/sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87
@@ -1 +1 @@
-variance(x) - Returns the variance of a set of numbers
\ No newline at end of file
+variance(x) - Returns the variance of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a b/sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a
index e17528e607abacbf88e1ca9d26e447eba05ce354..b14cf43bc0a95ad2c239ebb6934d7cfe1828f0f2 100644
--- a/sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a
+++ b/sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a
@@ -1,2 +1,2 @@
 variance(x) - Returns the variance of a set of numbers
-Synonyms: var_pop
\ No newline at end of file
+Synonyms: var_pop
diff --git a/sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de b/sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de
index 9dbf241e281074f3ef223b5e9e80a4828df179ed..79e293fd4ed52fd5edf276dc1b6ca0e335ab238a 100644
--- a/sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de
+++ b/sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de
@@ -1 +1 @@
-var_pop(x) - Returns the variance of a set of numbers
\ No newline at end of file
+var_pop(x) - Returns the variance of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766 b/sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766
index 234a65bd8255d2219f186bf6d08fa43a98becdfb..4b4de5db812e9c493c40795bc4df6aac00eb3cd3 100644
--- a/sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766
+++ b/sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766
@@ -1,2 +1,2 @@
 var_pop(x) - Returns the variance of a set of numbers
-Synonyms: variance
\ No newline at end of file
+Synonyms: variance
diff --git a/sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 b/sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87
index f58ae0769d5a9dc9a4350ad1df972b5bce6018fb..260300b7d3bd99b8eca4d3aadb512212e1e28756 100644
--- a/sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87
+++ b/sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87
@@ -1 +1 @@
-variance(x) - Returns the variance of a set of numbers
\ No newline at end of file
+variance(x) - Returns the variance of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a b/sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a
index e17528e607abacbf88e1ca9d26e447eba05ce354..b14cf43bc0a95ad2c239ebb6934d7cfe1828f0f2 100644
--- a/sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a
+++ b/sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a
@@ -1,2 +1,2 @@
 variance(x) - Returns the variance of a set of numbers
-Synonyms: var_pop
\ No newline at end of file
+Synonyms: var_pop
diff --git a/sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de b/sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de
index 9dbf241e281074f3ef223b5e9e80a4828df179ed..79e293fd4ed52fd5edf276dc1b6ca0e335ab238a 100644
--- a/sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de
+++ b/sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de
@@ -1 +1 @@
-var_pop(x) - Returns the variance of a set of numbers
\ No newline at end of file
+var_pop(x) - Returns the variance of a set of numbers
diff --git a/sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766 b/sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766
index 234a65bd8255d2219f186bf6d08fa43a98becdfb..4b4de5db812e9c493c40795bc4df6aac00eb3cd3 100644
--- a/sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766
+++ b/sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766
@@ -1,2 +1,2 @@
 var_pop(x) - Returns the variance of a set of numbers
-Synonyms: variance
\ No newline at end of file
+Synonyms: variance
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1 b/sql/hive/src/test/resources/golden/udf_weekofyear-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1
rename to sql/hive/src/test/resources/golden/udf_weekofyear-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 b/sql/hive/src/test/resources/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2
deleted file mode 100644
index 29e3b370b03b85c014dd2215c17a57b64a98f60f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2
+++ /dev/null
@@ -1 +0,0 @@
-weekofyear(date) - Returns the week of the year of the given date. A week is considered to start on a Monday and week 1 is the first week with >3 days.
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd b/sql/hive/src/test/resources/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd
deleted file mode 100644
index c7939a11937c507b4dcf54a39edee4d6d9082c35..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd
+++ /dev/null
@@ -1,6 +0,0 @@
-weekofyear(date) - Returns the week of the year of the given date. A week is considered to start on a Monday and week 1 is the first week with >3 days.
-Examples:
-  > SELECT weekofyear('2008-02-20') FROM src LIMIT 1;
-  8
-  > SELECT weekofyear('1980-12-31 12:59:59') FROM src LIMIT 1;
-  1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-1-d6b4490b549a358be375511e39627dc2 b/sql/hive/src/test/resources/golden/udf_weekofyear-1-d6b4490b549a358be375511e39627dc2
new file mode 100644
index 0000000000000000000000000000000000000000..dca04d210339a80723418d4250e66a763e591a26
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_weekofyear-1-d6b4490b549a358be375511e39627dc2
@@ -0,0 +1 @@
+weekofyear(date) - Returns the week of the year of the given date. A week is considered to start on a Monday and week 1 is the first week with >3 days.
diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 b/sql/hive/src/test/resources/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6
deleted file mode 100644
index 20d1dc50ce1f7e0dd532a2f33e4d6fdba9235eb1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6
+++ /dev/null
@@ -1 +0,0 @@
-1	1	2	1	52	8	52	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-2-b7bbdfabe6054a66701250fd70065ddd b/sql/hive/src/test/resources/golden/udf_weekofyear-2-b7bbdfabe6054a66701250fd70065ddd
new file mode 100644
index 0000000000000000000000000000000000000000..337257dabd8d1d163cf6468da81da1900e651829
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_weekofyear-2-b7bbdfabe6054a66701250fd70065ddd
@@ -0,0 +1,6 @@
+weekofyear(date) - Returns the week of the year of the given date. A week is considered to start on a Monday and week 1 is the first week with >3 days.
+Examples:
+  > SELECT weekofyear('2008-02-20') FROM src LIMIT 1;
+  8
+  > SELECT weekofyear('1980-12-31 12:59:59') FROM src LIMIT 1;
+  1
diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-3-d5dd3abb6c8c7046a85dd05f51126285 b/sql/hive/src/test/resources/golden/udf_weekofyear-3-d5dd3abb6c8c7046a85dd05f51126285
new file mode 100644
index 0000000000000000000000000000000000000000..9d0163569f7b375140b78562e2feb20d56ce152f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_weekofyear-3-d5dd3abb6c8c7046a85dd05f51126285
@@ -0,0 +1 @@
+1	1	2	1	52	8	52	1
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6 b/sql/hive/src/test/resources/golden/udf_when-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6
rename to sql/hive/src/test/resources/golden/udf_when-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 b/sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11
deleted file mode 100644
index 2e36162dbaa33538d2e44492aa7ae880f697a858..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11
+++ /dev/null
@@ -1 +0,0 @@
-There is no documentation for function 'when'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 b/sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4
deleted file mode 100644
index 2e36162dbaa33538d2e44492aa7ae880f697a858..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4
+++ /dev/null
@@ -1 +0,0 @@
-There is no documentation for function 'when'
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_when-1-88b97c6722176393e9b3d089559d2d11 b/sql/hive/src/test/resources/golden/udf_when-1-88b97c6722176393e9b3d089559d2d11
new file mode 100644
index 0000000000000000000000000000000000000000..3626da8447ca08063b31ff46369416a2da826e12
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_when-1-88b97c6722176393e9b3d089559d2d11
@@ -0,0 +1 @@
+There is no documentation for function 'when'
diff --git a/sql/hive/src/test/resources/golden/udf_when-2-7365d5fe39dc7a025c942dad8fb9f0d4 b/sql/hive/src/test/resources/golden/udf_when-2-7365d5fe39dc7a025c942dad8fb9f0d4
new file mode 100644
index 0000000000000000000000000000000000000000..3626da8447ca08063b31ff46369416a2da826e12
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_when-2-7365d5fe39dc7a025c942dad8fb9f0d4
@@ -0,0 +1 @@
+There is no documentation for function 'when'
diff --git a/sql/hive/src/test/resources/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 b/sql/hive/src/test/resources/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 b/sql/hive/src/test/resources/golden/udf_when-3-734890c41528b9d918db66b0582228a4
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2
rename to sql/hive/src/test/resources/golden/udf_when-3-734890c41528b9d918db66b0582228a4
diff --git a/sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f b/sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f
deleted file mode 100644
index 6929fdca28eb8c2c2515e131ebe2bb8b6ca2c150..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f
+++ /dev/null
@@ -1 +0,0 @@
-2	9	14	NULL	24	NULL
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_when-4-c57d6eb11efc29ce3a9c450488f3d750 b/sql/hive/src/test/resources/golden/udf_when-4-c57d6eb11efc29ce3a9c450488f3d750
new file mode 100644
index 0000000000000000000000000000000000000000..b28b0050514f3881974b90a19bae314ca8c79d7c
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_when-4-c57d6eb11efc29ce3a9c450488f3d750
@@ -0,0 +1 @@
+2	9	14	NULL	24	NULL
diff --git a/sql/hive/src/test/resources/golden/udf_when-5-6ed21e998c0fc32c39f6375136f55de6 b/sql/hive/src/test/resources/golden/udf_when-5-6ed21e998c0fc32c39f6375136f55de6
new file mode 100644
index 0000000000000000000000000000000000000000..db46b5a72aa80f25693fe12224c67ff280cec4cc
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_when-5-6ed21e998c0fc32c39f6375136f55de6
@@ -0,0 +1 @@
+123	123.0	abcd
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681 b/sql/hive/src/test/resources/golden/udf_xpath-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681
rename to sql/hive/src/test/resources/golden/udf_xpath-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 b/sql/hive/src/test/resources/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8
deleted file mode 100644
index 18d9720653a5b0cf347bd617c8ee4c5bd8ffa7ce..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8
+++ /dev/null
@@ -1 +0,0 @@
-xpath(xml, xpath) - Returns a string array of values within xml nodes that match the xpath expression
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 b/sql/hive/src/test/resources/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71
deleted file mode 100644
index 3b092dafd32c2310ec104e537fd56b30b0075fe9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71
+++ /dev/null
@@ -1,8 +0,0 @@
-xpath(xml, xpath) - Returns a string array of values within xml nodes that match the xpath expression
-Example:
-  > SELECT xpath('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>', 'a/text()') FROM src LIMIT 1
-  []
-  > SELECT xpath('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>', 'a/b/text()') FROM src LIMIT 1
-  ["b1","b2","b3"]
-  > SELECT xpath('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>', 'a/c/text()') FROM src LIMIT 1
-  ["c1","c2"]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath-1-622670bd1cbf4bf0cf04a553006e3c8 b/sql/hive/src/test/resources/golden/udf_xpath-1-622670bd1cbf4bf0cf04a553006e3c8
new file mode 100644
index 0000000000000000000000000000000000000000..d53b034f8fc6d152ce3ccbe4e1aa942909a83c39
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath-1-622670bd1cbf4bf0cf04a553006e3c8
@@ -0,0 +1 @@
+xpath(xml, xpath) - Returns a string array of values within xml nodes that match the xpath expression
diff --git a/sql/hive/src/test/resources/golden/udf_xpath-2-5a6f10392053a4eabe62f5cabb073a71 b/sql/hive/src/test/resources/golden/udf_xpath-2-5a6f10392053a4eabe62f5cabb073a71
new file mode 100644
index 0000000000000000000000000000000000000000..bff795649a7ea8177cc60c31a3749634db2efaf4
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath-2-5a6f10392053a4eabe62f5cabb073a71
@@ -0,0 +1,8 @@
+xpath(xml, xpath) - Returns a string array of values within xml nodes that match the xpath expression
+Example:
+  > SELECT xpath('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>', 'a/text()') FROM src LIMIT 1
+  []
+  > SELECT xpath('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>', 'a/b/text()') FROM src LIMIT 1
+  ["b1","b2","b3"]
+  > SELECT xpath('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>', 'a/c/text()') FROM src LIMIT 1
+  ["c1","c2"]
diff --git a/sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d b/sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d
deleted file mode 100644
index 0637a088a01e8ddab3bf3fa98dbe804cbde1a0dc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d
+++ /dev/null
@@ -1 +0,0 @@
-[]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea b/sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea
deleted file mode 100644
index 8e8aff2387621f1a303622d5afae57a5bdea9866..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea
+++ /dev/null
@@ -1 +0,0 @@
-["b1","b2","b3","c1","c2"]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath-3-f0b9adf99c68290c86d0f40f45111e18 b/sql/hive/src/test/resources/golden/udf_xpath-3-f0b9adf99c68290c86d0f40f45111e18
new file mode 100644
index 0000000000000000000000000000000000000000..fe51488c7066f6687ef680d6bfaa4f7768ef205c
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath-3-f0b9adf99c68290c86d0f40f45111e18
@@ -0,0 +1 @@
+[]
diff --git a/sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a b/sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a
deleted file mode 100644
index e518b1f37e124e7cfbaa8343de94b17d0269d4a5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a
+++ /dev/null
@@ -1 +0,0 @@
-["b1","b2","b3"]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath-4-6d5e7ed902ac051f0cdba43d7a30434e b/sql/hive/src/test/resources/golden/udf_xpath-4-6d5e7ed902ac051f0cdba43d7a30434e
new file mode 100644
index 0000000000000000000000000000000000000000..96e245a7eb37ca96b04fd414eff5074947a8b94e
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath-4-6d5e7ed902ac051f0cdba43d7a30434e
@@ -0,0 +1 @@
+["b1","b2","b3","c1","c2"]
diff --git a/sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 b/sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0
deleted file mode 100644
index 45615b772ba35a7b56d4f5512e7d0af6648c7eea..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0
+++ /dev/null
@@ -1 +0,0 @@
-["c1","c2"]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath-5-b66a64f91dd812fda2859863855988b5 b/sql/hive/src/test/resources/golden/udf_xpath-5-b66a64f91dd812fda2859863855988b5
new file mode 100644
index 0000000000000000000000000000000000000000..44def14ea2c68208c42de3b735852a32dd53fbd7
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath-5-b66a64f91dd812fda2859863855988b5
@@ -0,0 +1 @@
+["b1","b2","b3"]
diff --git a/sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f b/sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f
deleted file mode 100644
index 327bf3a1e76ccceba5da07fd830dfdaf69cedc85..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f
+++ /dev/null
@@ -1 +0,0 @@
-["b1","c1"]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath-6-d4f95ebc0057639138900722c74ee17a b/sql/hive/src/test/resources/golden/udf_xpath-6-d4f95ebc0057639138900722c74ee17a
new file mode 100644
index 0000000000000000000000000000000000000000..6a97cc328aa70994a49a25016420e5a9227af622
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath-6-d4f95ebc0057639138900722c74ee17a
@@ -0,0 +1 @@
+["c1","c2"]
diff --git a/sql/hive/src/test/resources/golden/udf_xpath-7-6b8fceac3bd654662f067da60670e1d9 b/sql/hive/src/test/resources/golden/udf_xpath-7-6b8fceac3bd654662f067da60670e1d9
new file mode 100644
index 0000000000000000000000000000000000000000..8b4abae5acff8e0f2f2b8be9f1c8b95be8842b4b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath-7-6b8fceac3bd654662f067da60670e1d9
@@ -0,0 +1 @@
+["b1","c1"]
diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7
rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371
deleted file mode 100644
index 134e9e99b7423c730fa0f238e4a90df5586b454f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371
+++ /dev/null
@@ -1 +0,0 @@
-xpath_boolean(xml, xpath) - Evaluates a boolean xpath expression
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-8b2f3f8132bfc8344e243cdaf45eb371 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-8b2f3f8132bfc8344e243cdaf45eb371
new file mode 100644
index 0000000000000000000000000000000000000000..d47e3fe7d34a153e7ac600a68578c224c61a087f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-8b2f3f8132bfc8344e243cdaf45eb371
@@ -0,0 +1 @@
+xpath_boolean(xml, xpath) - Evaluates a boolean xpath expression
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4
deleted file mode 100644
index 624889cc0ac672d98353f608e6e6f633cff8b35c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4
+++ /dev/null
@@ -1,6 +0,0 @@
-xpath_boolean(xml, xpath) - Evaluates a boolean xpath expression
-Example:
-  > SELECT xpath_boolean('<a><b>1</b></a>','a/b') FROM src LIMIT 1;
-  true
-  > SELECT xpath_boolean('<a><b>1</b></a>','a/b = 2') FROM src LIMIT 1;
-  false
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b b/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b
deleted file mode 100644
index f32a5804e292d30bedf68f62d32fb75d87e99fd9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b
+++ /dev/null
@@ -1 +0,0 @@
-true
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-9e935539c9a3f3a118145096e7f978c4 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-9e935539c9a3f3a118145096e7f978c4
new file mode 100644
index 0000000000000000000000000000000000000000..4f3c2652aad112e5a940f56789299d8cdc61de33
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-9e935539c9a3f3a118145096e7f978c4
@@ -0,0 +1,6 @@
+xpath_boolean(xml, xpath) - Evaluates a boolean xpath expression
+Example:
+  > SELECT xpath_boolean('<a><b>1</b></a>','a/b') FROM src LIMIT 1;
+  true
+  > SELECT xpath_boolean('<a><b>1</b></a>','a/b = 2') FROM src LIMIT 1;
+  false
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f b/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-9b97a08303a7a89e7575687f6d7ba435
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f
rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-3-9b97a08303a7a89e7575687f6d7ba435
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060
deleted file mode 100644
index 02e4a84d62c4b0fe9cca60bba7b9799f78f1f7ed..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060
+++ /dev/null
@@ -1 +0,0 @@
-false
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df b/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-284ce7d6fc6850ca853111145784286b
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df
rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-4-284ce7d6fc6850ca853111145784286b
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965
deleted file mode 100644
index f32a5804e292d30bedf68f62d32fb75d87e99fd9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965
+++ /dev/null
@@ -1 +0,0 @@
-true
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a b/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-95a0dfa897ba9050ad751a78aeb72f3d
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a
rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-5-95a0dfa897ba9050ad751a78aeb72f3d
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302
deleted file mode 100644
index 02e4a84d62c4b0fe9cca60bba7b9799f78f1f7ed..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302
+++ /dev/null
@@ -1 +0,0 @@
-false
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f b/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f
deleted file mode 100644
index 02e4a84d62c4b0fe9cca60bba7b9799f78f1f7ed..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f
+++ /dev/null
@@ -1 +0,0 @@
-false
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-9-be623247e4dbf119b43458b72d1be017 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-796c717c9d52d3efa374d12fe65259e6
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-9-be623247e4dbf119b43458b72d1be017
rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-6-796c717c9d52d3efa374d12fe65259e6
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303
deleted file mode 100644
index f32a5804e292d30bedf68f62d32fb75d87e99fd9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303
+++ /dev/null
@@ -1 +0,0 @@
-true
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-cc3ae9a7505e04a2e9b950442a81a559 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-cc3ae9a7505e04a2e9b950442a81a559
new file mode 100644
index 0000000000000000000000000000000000000000..c508d5366f70bba37fcc09d128b6537c4adb2c79
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-cc3ae9a7505e04a2e9b950442a81a559
@@ -0,0 +1 @@
+false
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-8-37229f303635a030f6cab20e0381f51f b/sql/hive/src/test/resources/golden/udf_xpath_boolean-8-62a1af222d4e12c59cef71b979c6e58
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_to_boolean-8-37229f303635a030f6cab20e0381f51f
rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-8-62a1af222d4e12c59cef71b979c6e58
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 b/sql/hive/src/test/resources/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67
deleted file mode 100644
index b56f4d00a8a74fb31bf5a0a2960c2e4122f489e3..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67
+++ /dev/null
@@ -1 +0,0 @@
-xpath_number(xml, xpath) - Returns a double value that matches the xpath expression
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_xpath_double-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe
rename to sql/hive/src/test/resources/golden/udf_xpath_double-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 b/sql/hive/src/test/resources/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2
deleted file mode 100644
index b6b62271746477a77e0737927f6ece6be75f25ab..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2
+++ /dev/null
@@ -1,5 +0,0 @@
-xpath_number(xml, xpath) - Returns a double value that matches the xpath expression
-Synonyms: xpath_double
-Example:
-  > SELECT xpath_number('<a><b>1</b><b>2</b></a>','sum(a/b)') FROM src LIMIT 1;
-  3.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-1-39199612969071d58b24034a2d17ca67 b/sql/hive/src/test/resources/golden/udf_xpath_double-1-39199612969071d58b24034a2d17ca67
new file mode 100644
index 0000000000000000000000000000000000000000..a9cb2ecb60e2834082015407a4e056b41c610bae
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_double-1-39199612969071d58b24034a2d17ca67
@@ -0,0 +1 @@
+xpath_number(xml, xpath) - Returns a double value that matches the xpath expression
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-10-ad0f9117f6f52590d38e619e998a6648 b/sql/hive/src/test/resources/golden/udf_xpath_double-10-ad0f9117f6f52590d38e619e998a6648
new file mode 100644
index 0000000000000000000000000000000000000000..fe6b09a7d14ccc307ab278380b4c9506395755ac
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_double-10-ad0f9117f6f52590d38e619e998a6648
@@ -0,0 +1 @@
+15.0
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 b/sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0
deleted file mode 100644
index 7104585f9a4397cb36585194da43b8922128bf8f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0
+++ /dev/null
@@ -1 +0,0 @@
-7.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 b/sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2
deleted file mode 100644
index 6e6366051638f977cca4368393347c872a05baa1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2
+++ /dev/null
@@ -1 +0,0 @@
-5.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-11-bf1f96ce71879c17d91ac9df44c36d29 b/sql/hive/src/test/resources/golden/udf_xpath_double-11-bf1f96ce71879c17d91ac9df44c36d29
new file mode 100644
index 0000000000000000000000000000000000000000..4fedf1d20e15761409d6e4e3bf99c0beb499fdfe
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_double-11-bf1f96ce71879c17d91ac9df44c36d29
@@ -0,0 +1 @@
+7.0
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-12-9621353ce5c583ca34216d357c5bb0eb b/sql/hive/src/test/resources/golden/udf_xpath_double-12-9621353ce5c583ca34216d357c5bb0eb
new file mode 100644
index 0000000000000000000000000000000000000000..819e07a22435f1e8efcbdd1d1c062deef0e501b1
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_double-12-9621353ce5c583ca34216d357c5bb0eb
@@ -0,0 +1 @@
+5.0
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-2-2d5ceab0a14d6e837ed153e1a2235bb2 b/sql/hive/src/test/resources/golden/udf_xpath_double-2-2d5ceab0a14d6e837ed153e1a2235bb2
new file mode 100644
index 0000000000000000000000000000000000000000..0f303917eb1bd1a41a944319d4fe88cc8544345e
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_double-2-2d5ceab0a14d6e837ed153e1a2235bb2
@@ -0,0 +1,5 @@
+xpath_number(xml, xpath) - Returns a double value that matches the xpath expression
+Synonyms: xpath_double
+Example:
+  > SELECT xpath_number('<a><b>1</b><b>2</b></a>','sum(a/b)') FROM src LIMIT 1;
+  3.0
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 b/sql/hive/src/test/resources/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789
deleted file mode 100644
index 5b6de7db2f9f3a60b5ca20b7aa1fbb8b37f9cb1c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789
+++ /dev/null
@@ -1 +0,0 @@
-xpath_double(xml, xpath) - Returns a double value that matches the xpath expression
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-3-a4d22cea9dffaf26b485b3eb08963789 b/sql/hive/src/test/resources/golden/udf_xpath_double-3-a4d22cea9dffaf26b485b3eb08963789
new file mode 100644
index 0000000000000000000000000000000000000000..20ff7713a919d9197d59fb34e08370c515cda4e8
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_double-3-a4d22cea9dffaf26b485b3eb08963789
@@ -0,0 +1 @@
+xpath_double(xml, xpath) - Returns a double value that matches the xpath expression
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 b/sql/hive/src/test/resources/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2
deleted file mode 100644
index b8accbcae1e176a7865682fd6e75706c3c6f865a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2
+++ /dev/null
@@ -1,5 +0,0 @@
-xpath_double(xml, xpath) - Returns a double value that matches the xpath expression
-Synonyms: xpath_number
-Example:
-  > SELECT xpath_double('<a><b>1</b><b>2</b></a>','sum(a/b)') FROM src LIMIT 1;
-  3.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c b/sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c
deleted file mode 100644
index b955cad2a39a09a1c4143ccffb078ac94bb80b4c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c
+++ /dev/null
@@ -1 +0,0 @@
-NaN
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-4-ea050c5b9e74d4b87b037236ef9e4fc2 b/sql/hive/src/test/resources/golden/udf_xpath_double-4-ea050c5b9e74d4b87b037236ef9e4fc2
new file mode 100644
index 0000000000000000000000000000000000000000..89c174f371b881447e6f2f2770b0d07ea717ca26
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_double-4-ea050c5b9e74d4b87b037236ef9e4fc2
@@ -0,0 +1,5 @@
+xpath_double(xml, xpath) - Returns a double value that matches the xpath expression
+Synonyms: xpath_number
+Example:
+  > SELECT xpath_double('<a><b>1</b><b>2</b></a>','sum(a/b)') FROM src LIMIT 1;
+  3.0
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-5-1e0514d71f99da09e01a414a4e01d046 b/sql/hive/src/test/resources/golden/udf_xpath_double-5-1e0514d71f99da09e01a414a4e01d046
new file mode 100644
index 0000000000000000000000000000000000000000..736991a13874574828c7a0fdd2ead5aac88cac65
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_double-5-1e0514d71f99da09e01a414a4e01d046
@@ -0,0 +1 @@
+NaN
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 b/sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5
deleted file mode 100644
index b955cad2a39a09a1c4143ccffb078ac94bb80b4c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5
+++ /dev/null
@@ -1 +0,0 @@
-NaN
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 b/sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6
deleted file mode 100644
index 0a8ac8c6295409ebe66ff33f00a61b54c3314a1d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6
+++ /dev/null
@@ -1 +0,0 @@
-8.0E19
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-6-6a3985167fedd579f7bd5745133a3524 b/sql/hive/src/test/resources/golden/udf_xpath_double-6-6a3985167fedd579f7bd5745133a3524
new file mode 100644
index 0000000000000000000000000000000000000000..736991a13874574828c7a0fdd2ead5aac88cac65
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_double-6-6a3985167fedd579f7bd5745133a3524
@@ -0,0 +1 @@
+NaN
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c b/sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c
deleted file mode 100644
index 171538eb0b00f4eddffa17929796de55b838f34b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c
+++ /dev/null
@@ -1 +0,0 @@
-0.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-7-d97e93fb4b69522300f505e04b6674c8 b/sql/hive/src/test/resources/golden/udf_xpath_double-7-d97e93fb4b69522300f505e04b6674c8
new file mode 100644
index 0000000000000000000000000000000000000000..f65000a36a478eabb2a9b4d6f6f110357811fd39
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_double-7-d97e93fb4b69522300f505e04b6674c8
@@ -0,0 +1 @@
+8.0E19
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d b/sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d
deleted file mode 100644
index 9f8e9b69a33f4e8067d5b21661a35d8856758aba..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d
+++ /dev/null
@@ -1 +0,0 @@
-1.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-8-fce6cafa07b75c9843c1c1964e84fc10 b/sql/hive/src/test/resources/golden/udf_xpath_double-8-fce6cafa07b75c9843c1c1964e84fc10
new file mode 100644
index 0000000000000000000000000000000000000000..ba66466c2a0d0a1080ad71d8e1a67a97f336c384
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_double-8-fce6cafa07b75c9843c1c1964e84fc10
@@ -0,0 +1 @@
+0.0
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e b/sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e
deleted file mode 100644
index 3d3be3c32ed853ef321465a16410dcd3377d07b9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e
+++ /dev/null
@@ -1 +0,0 @@
-15.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-9-f27361521c35bf4f6581dba4c185d550 b/sql/hive/src/test/resources/golden/udf_xpath_double-9-f27361521c35bf4f6581dba4c185d550
new file mode 100644
index 0000000000000000000000000000000000000000..d3827e75a5cadb9fe4a27e1cb9b6d192e7323120
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_double-9-f27361521c35bf4f6581dba4c185d550
@@ -0,0 +1 @@
+1.0
diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 b/sql/hive/src/test/resources/golden/udf_xpath_float-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412
rename to sql/hive/src/test/resources/golden/udf_xpath_float-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 b/sql/hive/src/test/resources/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791
deleted file mode 100644
index ea4df0ae58e12d6f82acd331560168624d0f7882..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791
+++ /dev/null
@@ -1 +0,0 @@
-xpath_float(xml, xpath) - Returns a float value that matches the xpath expression
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-1-7483dafee0dc7334eecabba31977e791 b/sql/hive/src/test/resources/golden/udf_xpath_float-1-7483dafee0dc7334eecabba31977e791
new file mode 100644
index 0000000000000000000000000000000000000000..b7dea42c782122f57a798d3f773e032a93c5c25f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_float-1-7483dafee0dc7334eecabba31977e791
@@ -0,0 +1 @@
+xpath_float(xml, xpath) - Returns a float value that matches the xpath expression
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a b/sql/hive/src/test/resources/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a
deleted file mode 100644
index 6bc4d4a46de72cda286af08757e04d5f14f83826..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a
+++ /dev/null
@@ -1,4 +0,0 @@
-xpath_float(xml, xpath) - Returns a float value that matches the xpath expression
-Example:
-  > SELECT xpath_float('<a><b>1</b><b>2</b></a>','sum(a/b)') FROM src LIMIT 1;
-  3.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-10-89ae28cf8e6b6f422d307a0085774cab b/sql/hive/src/test/resources/golden/udf_xpath_float-10-89ae28cf8e6b6f422d307a0085774cab
new file mode 100644
index 0000000000000000000000000000000000000000..819e07a22435f1e8efcbdd1d1c062deef0e501b1
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_float-10-89ae28cf8e6b6f422d307a0085774cab
@@ -0,0 +1 @@
+5.0
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 b/sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16
deleted file mode 100644
index b955cad2a39a09a1c4143ccffb078ac94bb80b4c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16
+++ /dev/null
@@ -1 +0,0 @@
-NaN
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-2-f6ddff2daba006787aeb861ca6f9d37a b/sql/hive/src/test/resources/golden/udf_xpath_float-2-f6ddff2daba006787aeb861ca6f9d37a
new file mode 100644
index 0000000000000000000000000000000000000000..02b139e7a3610688eddc1ffbadfae763b36c1947
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_float-2-f6ddff2daba006787aeb861ca6f9d37a
@@ -0,0 +1,4 @@
+xpath_float(xml, xpath) - Returns a float value that matches the xpath expression
+Example:
+  > SELECT xpath_float('<a><b>1</b><b>2</b></a>','sum(a/b)') FROM src LIMIT 1;
+  3.0
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 b/sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54
deleted file mode 100644
index b955cad2a39a09a1c4143ccffb078ac94bb80b4c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54
+++ /dev/null
@@ -1 +0,0 @@
-NaN
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-3-b743a9cb9f8688220e0a6346856f18ce b/sql/hive/src/test/resources/golden/udf_xpath_float-3-b743a9cb9f8688220e0a6346856f18ce
new file mode 100644
index 0000000000000000000000000000000000000000..736991a13874574828c7a0fdd2ead5aac88cac65
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_float-3-b743a9cb9f8688220e0a6346856f18ce
@@ -0,0 +1 @@
+NaN
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-4-6720ee0163b0f0ddd2ab72fa9ab935e3 b/sql/hive/src/test/resources/golden/udf_xpath_float-4-6720ee0163b0f0ddd2ab72fa9ab935e3
new file mode 100644
index 0000000000000000000000000000000000000000..736991a13874574828c7a0fdd2ead5aac88cac65
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_float-4-6720ee0163b0f0ddd2ab72fa9ab935e3
@@ -0,0 +1 @@
+NaN
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 b/sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352
deleted file mode 100644
index 0a8ac8c6295409ebe66ff33f00a61b54c3314a1d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352
+++ /dev/null
@@ -1 +0,0 @@
-8.0E19
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f b/sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f
deleted file mode 100644
index 171538eb0b00f4eddffa17929796de55b838f34b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f
+++ /dev/null
@@ -1 +0,0 @@
-0.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-5-f5e1100f6e8de31081042413b4039fb2 b/sql/hive/src/test/resources/golden/udf_xpath_float-5-f5e1100f6e8de31081042413b4039fb2
new file mode 100644
index 0000000000000000000000000000000000000000..f65000a36a478eabb2a9b4d6f6f110357811fd39
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_float-5-f5e1100f6e8de31081042413b4039fb2
@@ -0,0 +1 @@
+8.0E19
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-6-5e8457209d15467af7f14c09dfadb894 b/sql/hive/src/test/resources/golden/udf_xpath_float-6-5e8457209d15467af7f14c09dfadb894
new file mode 100644
index 0000000000000000000000000000000000000000..ba66466c2a0d0a1080ad71d8e1a67a97f336c384
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_float-6-5e8457209d15467af7f14c09dfadb894
@@ -0,0 +1 @@
+0.0
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf b/sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf
deleted file mode 100644
index 9f8e9b69a33f4e8067d5b21661a35d8856758aba..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf
+++ /dev/null
@@ -1 +0,0 @@
-1.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-7-580ffe4dabef758c4fcb52050b315615 b/sql/hive/src/test/resources/golden/udf_xpath_float-7-580ffe4dabef758c4fcb52050b315615
new file mode 100644
index 0000000000000000000000000000000000000000..d3827e75a5cadb9fe4a27e1cb9b6d192e7323120
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_float-7-580ffe4dabef758c4fcb52050b315615
@@ -0,0 +1 @@
+1.0
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa b/sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa
deleted file mode 100644
index 3d3be3c32ed853ef321465a16410dcd3377d07b9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa
+++ /dev/null
@@ -1 +0,0 @@
-15.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 b/sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48
deleted file mode 100644
index 7104585f9a4397cb36585194da43b8922128bf8f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48
+++ /dev/null
@@ -1 +0,0 @@
-7.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-8-73cfa6fe399ca3e240b84b47ee4f8bc8 b/sql/hive/src/test/resources/golden/udf_xpath_float-8-73cfa6fe399ca3e240b84b47ee4f8bc8
new file mode 100644
index 0000000000000000000000000000000000000000..fe6b09a7d14ccc307ab278380b4c9506395755ac
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_float-8-73cfa6fe399ca3e240b84b47ee4f8bc8
@@ -0,0 +1 @@
+15.0
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 b/sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401
deleted file mode 100644
index 6e6366051638f977cca4368393347c872a05baa1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401
+++ /dev/null
@@ -1 +0,0 @@
-5.0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-9-bf66b884da717d46f0b12c3b8cf8313a b/sql/hive/src/test/resources/golden/udf_xpath_float-9-bf66b884da717d46f0b12c3b8cf8313a
new file mode 100644
index 0000000000000000000000000000000000000000..4fedf1d20e15761409d6e4e3bf99c0beb499fdfe
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_float-9-bf66b884da717d46f0b12c3b8cf8313a
@@ -0,0 +1 @@
+7.0
diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 b/sql/hive/src/test/resources/golden/udf_xpath_int-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20
rename to sql/hive/src/test/resources/golden/udf_xpath_int-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 b/sql/hive/src/test/resources/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12
deleted file mode 100644
index 67ab19492e40591db8ea4699a9348ed9c121569f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12
+++ /dev/null
@@ -1 +0,0 @@
-xpath_int(xml, xpath) - Returns an integer value that matches the xpath expression
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 b/sql/hive/src/test/resources/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910
deleted file mode 100644
index 33349b0b22ad95c83f8de75ede11c7016eaf0ef2..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910
+++ /dev/null
@@ -1,4 +0,0 @@
-xpath_int(xml, xpath) - Returns an integer value that matches the xpath expression
-Example:
-  > SELECT xpath_int('<a><b>1</b><b>2</b></a>','sum(a/b)') FROM src LIMIT 1;
-  3
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-1-a9ed102a70e7e9d505be1555f0c7eb12 b/sql/hive/src/test/resources/golden/udf_xpath_int-1-a9ed102a70e7e9d505be1555f0c7eb12
new file mode 100644
index 0000000000000000000000000000000000000000..000f90d69e7f73b818b18ff10eb300273792d485
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_int-1-a9ed102a70e7e9d505be1555f0c7eb12
@@ -0,0 +1 @@
+xpath_int(xml, xpath) - Returns an integer value that matches the xpath expression
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-10-ea25feb474c40c6985152093d4dbb13a b/sql/hive/src/test/resources/golden/udf_xpath_int-10-ea25feb474c40c6985152093d4dbb13a
new file mode 100644
index 0000000000000000000000000000000000000000..7ed6ff82de6bcc2a78243fc9c54d3ef5ac14da69
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_int-10-ea25feb474c40c6985152093d4dbb13a
@@ -0,0 +1 @@
+5
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-2-26ddf7e46a66065909e9e547f0459910 b/sql/hive/src/test/resources/golden/udf_xpath_int-2-26ddf7e46a66065909e9e547f0459910
new file mode 100644
index 0000000000000000000000000000000000000000..f969b9a48d10ffceef23d4651c7ef9be2b4e1231
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_int-2-26ddf7e46a66065909e9e547f0459910
@@ -0,0 +1,4 @@
+xpath_int(xml, xpath) - Returns an integer value that matches the xpath expression
+Example:
+  > SELECT xpath_int('<a><b>1</b><b>2</b></a>','sum(a/b)') FROM src LIMIT 1;
+  3
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba b/sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_xpath_int-3-a04ed6832ab542d6ee5903039511a826
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7
rename to sql/hive/src/test/resources/golden/udf_xpath_int-3-a04ed6832ab542d6ee5903039511a826
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba b/sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 b/sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890
deleted file mode 100644
index 55a89f5564d21f4ed36ca1c90fc046f28b8cd604..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890
+++ /dev/null
@@ -1 +0,0 @@
-2147483647
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_xpath_int-4-bf5a4dbb7a98abc91111a3798b56809f
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe
rename to sql/hive/src/test/resources/golden/udf_xpath_int-4-bf5a4dbb7a98abc91111a3798b56809f
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 b/sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-5-f49db0ecb889722ec68d1753c084b0e1 b/sql/hive/src/test/resources/golden/udf_xpath_int-5-f49db0ecb889722ec68d1753c084b0e1
new file mode 100644
index 0000000000000000000000000000000000000000..a51fa7d1efef69aed6609d501159f7b16c21d7ab
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_int-5-f49db0ecb889722ec68d1753c084b0e1
@@ -0,0 +1 @@
+2147483647
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a b/sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a
deleted file mode 100644
index 56a6051ca2b02b04ef92d5150c9ef600403cb1de..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a
+++ /dev/null
@@ -1 +0,0 @@
-1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/udf_xpath_int-6-ac509f06f01c02924adef220404fc515
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf
rename to sql/hive/src/test/resources/golden/udf_xpath_int-6-ac509f06f01c02924adef220404fc515
diff --git a/sql/hive/src/test/resources/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d b/sql/hive/src/test/resources/golden/udf_xpath_int-7-87ff12d650afb8f511d906778450fae7
similarity index 100%
rename from sql/hive/src/test/resources/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d
rename to sql/hive/src/test/resources/golden/udf_xpath_int-7-87ff12d650afb8f511d906778450fae7
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 b/sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4
deleted file mode 100644
index 3f10ffe7a4c473619c926cfb1e8d95e726e5a0ec..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4
+++ /dev/null
@@ -1 +0,0 @@
-15
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-8-a175811eca252aa77c16a44fbb0ee7b2 b/sql/hive/src/test/resources/golden/udf_xpath_int-8-a175811eca252aa77c16a44fbb0ee7b2
new file mode 100644
index 0000000000000000000000000000000000000000..60d3b2f4a4cd5f1637eba020358bfe5ecb5edcf2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_int-8-a175811eca252aa77c16a44fbb0ee7b2
@@ -0,0 +1 @@
+15
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 b/sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833
deleted file mode 100644
index c7930257dfef505fd996e1d6f22f2f35149990d0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833
+++ /dev/null
@@ -1 +0,0 @@
-7
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-9-7da88f589199f5ca873780fb22614573 b/sql/hive/src/test/resources/golden/udf_xpath_int-9-7da88f589199f5ca873780fb22614573
new file mode 100644
index 0000000000000000000000000000000000000000..7f8f011eb73d6043d2e6db9d2c101195ae2801f2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_int-9-7da88f589199f5ca873780fb22614573
@@ -0,0 +1 @@
+7
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 b/sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869
deleted file mode 100644
index 7813681f5b41c028345ca62a2be376bae70b7f61..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869
+++ /dev/null
@@ -1 +0,0 @@
-5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/udf_xpath_long-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf
rename to sql/hive/src/test/resources/golden/udf_xpath_long-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b b/sql/hive/src/test/resources/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b
deleted file mode 100644
index 748fd827e8545693a72786695b791ab71f384d7a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b
+++ /dev/null
@@ -1 +0,0 @@
-xpath_long(xml, xpath) - Returns a long value that matches the xpath expression
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 b/sql/hive/src/test/resources/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6
deleted file mode 100644
index 65241b6ce9b64546a589519fdc3a17e8c045034e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6
+++ /dev/null
@@ -1,4 +0,0 @@
-xpath_long(xml, xpath) - Returns a long value that matches the xpath expression
-Example:
-  > SELECT xpath_long('<a><b>1</b><b>2</b></a>','sum(a/b)') FROM src LIMIT 1;
-  3
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-1-d274b272251e95ec2e8139bfa471bd0b b/sql/hive/src/test/resources/golden/udf_xpath_long-1-d274b272251e95ec2e8139bfa471bd0b
new file mode 100644
index 0000000000000000000000000000000000000000..ee1a589d6652b5a926531c34f01563cefd5a2b13
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_long-1-d274b272251e95ec2e8139bfa471bd0b
@@ -0,0 +1 @@
+xpath_long(xml, xpath) - Returns a long value that matches the xpath expression
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-10-caeac94758a40493a5227fcdb8ec2f87 b/sql/hive/src/test/resources/golden/udf_xpath_long-10-caeac94758a40493a5227fcdb8ec2f87
new file mode 100644
index 0000000000000000000000000000000000000000..7ed6ff82de6bcc2a78243fc9c54d3ef5ac14da69
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_long-10-caeac94758a40493a5227fcdb8ec2f87
@@ -0,0 +1 @@
+5
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-2-43fbf4d944cf7eaf57b4d6166b803fb6 b/sql/hive/src/test/resources/golden/udf_xpath_long-2-43fbf4d944cf7eaf57b4d6166b803fb6
new file mode 100644
index 0000000000000000000000000000000000000000..b9eb5a4a99c02739e8ec187a2ee8b9194f7c5684
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_long-2-43fbf4d944cf7eaf57b4d6166b803fb6
@@ -0,0 +1,4 @@
+xpath_long(xml, xpath) - Returns a long value that matches the xpath expression
+Example:
+  > SELECT xpath_long('<a><b>1</b><b>2</b></a>','sum(a/b)') FROM src LIMIT 1;
+  3
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b b/sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/udf_xpath_long-3-9df8d27e31d96e0b35b9b40910d4bd98
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf
rename to sql/hive/src/test/resources/golden/udf_xpath_long-3-9df8d27e31d96e0b35b9b40910d4bd98
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d b/sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/udf_xpath_long-4-3211913c56521887d30e3d1a50762b3f
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0
rename to sql/hive/src/test/resources/golden/udf_xpath_long-4-3211913c56521887d30e3d1a50762b3f
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 b/sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9
deleted file mode 100644
index 996d127e593651551ed0d8c6b3e893e1e36eaff8..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9
+++ /dev/null
@@ -1 +0,0 @@
-9223372036854775807
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d b/sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-5-d580a8f01a546baddd939b95722e6354 b/sql/hive/src/test/resources/golden/udf_xpath_long-5-d580a8f01a546baddd939b95722e6354
new file mode 100644
index 0000000000000000000000000000000000000000..2045006edaf5e046dcad35b747423cfe50c92c45
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_long-5-d580a8f01a546baddd939b95722e6354
@@ -0,0 +1 @@
+9223372036854775807
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/udf_xpath_long-6-b695348ed3faec63be2c07d0d4afaaf3
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717
rename to sql/hive/src/test/resources/golden/udf_xpath_long-6-b695348ed3faec63be2c07d0d4afaaf3
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 b/sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787
deleted file mode 100644
index 56a6051ca2b02b04ef92d5150c9ef600403cb1de..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787
+++ /dev/null
@@ -1 +0,0 @@
-1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 b/sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0
deleted file mode 100644
index 3f10ffe7a4c473619c926cfb1e8d95e726e5a0ec..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0
+++ /dev/null
@@ -1 +0,0 @@
-15
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc b/sql/hive/src/test/resources/golden/udf_xpath_long-7-ed5af6d7451107a753b2c7ff130ac73b
similarity index 100%
rename from sql/hive/src/test/resources/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc
rename to sql/hive/src/test/resources/golden/udf_xpath_long-7-ed5af6d7451107a753b2c7ff130ac73b
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-8-2f952b56682969bb203fa3d9102f7015 b/sql/hive/src/test/resources/golden/udf_xpath_long-8-2f952b56682969bb203fa3d9102f7015
new file mode 100644
index 0000000000000000000000000000000000000000..60d3b2f4a4cd5f1637eba020358bfe5ecb5edcf2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_long-8-2f952b56682969bb203fa3d9102f7015
@@ -0,0 +1 @@
+15
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 b/sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083
deleted file mode 100644
index c7930257dfef505fd996e1d6f22f2f35149990d0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083
+++ /dev/null
@@ -1 +0,0 @@
-7
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 b/sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1
deleted file mode 100644
index 7813681f5b41c028345ca62a2be376bae70b7f61..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1
+++ /dev/null
@@ -1 +0,0 @@
-5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-9-947b6e08ba9c7defd75d00412f9bc4fd b/sql/hive/src/test/resources/golden/udf_xpath_long-9-947b6e08ba9c7defd75d00412f9bc4fd
new file mode 100644
index 0000000000000000000000000000000000000000..7f8f011eb73d6043d2e6db9d2c101195ae2801f2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_long-9-947b6e08ba9c7defd75d00412f9bc4fd
@@ -0,0 +1 @@
+7
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/udf_xpath_short-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f
rename to sql/hive/src/test/resources/golden/udf_xpath_short-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 b/sql/hive/src/test/resources/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17
deleted file mode 100644
index 22ccb9ca9defa423460a1559a709754ce25464a4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17
+++ /dev/null
@@ -1 +0,0 @@
-xpath_short(xml, xpath) - Returns a short value that matches the xpath expression
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 b/sql/hive/src/test/resources/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2
deleted file mode 100644
index 931ce2abb3d4dd8e4ffa709286b5801b00016ed5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2
+++ /dev/null
@@ -1,4 +0,0 @@
-xpath_short(xml, xpath) - Returns a short value that matches the xpath expression
-Example:
-  > SELECT xpath_short('<a><b>1</b><b>2</b></a>','sum(a/b)') FROM src LIMIT 1;
-  3
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-1-7d5231aed9cbbf68cd740791f9e5be17 b/sql/hive/src/test/resources/golden/udf_xpath_short-1-7d5231aed9cbbf68cd740791f9e5be17
new file mode 100644
index 0000000000000000000000000000000000000000..63ab4f15e8c84a913a02a1b4f20557ede4c1df96
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_short-1-7d5231aed9cbbf68cd740791f9e5be17
@@ -0,0 +1 @@
+xpath_short(xml, xpath) - Returns a short value that matches the xpath expression
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-10-b537709676634250e13914e76cd9a530 b/sql/hive/src/test/resources/golden/udf_xpath_short-10-b537709676634250e13914e76cd9a530
new file mode 100644
index 0000000000000000000000000000000000000000..7ed6ff82de6bcc2a78243fc9c54d3ef5ac14da69
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_short-10-b537709676634250e13914e76cd9a530
@@ -0,0 +1 @@
+5
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 b/sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-2-5d45932563b78e0b435b8cfebfe3cc2 b/sql/hive/src/test/resources/golden/udf_xpath_short-2-5d45932563b78e0b435b8cfebfe3cc2
new file mode 100644
index 0000000000000000000000000000000000000000..ecb5e9a83109c18a8a1f5a63658dee3c42f58faa
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_short-2-5d45932563b78e0b435b8cfebfe3cc2
@@ -0,0 +1,4 @@
+xpath_short(xml, xpath) - Returns a short value that matches the xpath expression
+Example:
+  > SELECT xpath_short('<a><b>1</b><b>2</b></a>','sum(a/b)') FROM src LIMIT 1;
+  3
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 b/sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/udf_xpath_short-3-c09b4ae6886fa58dcdd728bef45e7efa
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f
rename to sql/hive/src/test/resources/golden/udf_xpath_short-3-c09b4ae6886fa58dcdd728bef45e7efa
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/udf_xpath_short-4-84f5f6bb2fdc0987d281d52a53a4b24e
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571
rename to sql/hive/src/test/resources/golden/udf_xpath_short-4-84f5f6bb2fdc0987d281d52a53a4b24e
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c b/sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c
deleted file mode 100644
index d7d17fcbef95ca19081c4cc5e97cbc592cc7081f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c
+++ /dev/null
@@ -1 +0,0 @@
--1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e b/sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/udf_xpath_short-5-c09fd0565ed041c773fee9bd0436e861
similarity index 100%
rename from sql/hive/src/test/resources/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07
rename to sql/hive/src/test/resources/golden/udf_xpath_short-5-c09fd0565ed041c773fee9bd0436e861
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/udf_xpath_short-6-16ced3de15d4ec87a4e7001376551758
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147
rename to sql/hive/src/test/resources/golden/udf_xpath_short-6-16ced3de15d4ec87a4e7001376551758
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 b/sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04
deleted file mode 100644
index 56a6051ca2b02b04ef92d5150c9ef600403cb1de..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04
+++ /dev/null
@@ -1 +0,0 @@
-1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/newline-0-43392a20a8d249a279d50d96578e6a1b b/sql/hive/src/test/resources/golden/udf_xpath_short-7-8ffdf20c15f3ed81bb5a92c61d200ae2
similarity index 100%
rename from sql/hive/src/test/resources/golden/newline-0-43392a20a8d249a279d50d96578e6a1b
rename to sql/hive/src/test/resources/golden/udf_xpath_short-7-8ffdf20c15f3ed81bb5a92c61d200ae2
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d b/sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d
deleted file mode 100644
index 3f10ffe7a4c473619c926cfb1e8d95e726e5a0ec..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d
+++ /dev/null
@@ -1 +0,0 @@
-15
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-8-102ad2dea8d94528b402d980a45d53d4 b/sql/hive/src/test/resources/golden/udf_xpath_short-8-102ad2dea8d94528b402d980a45d53d4
new file mode 100644
index 0000000000000000000000000000000000000000..60d3b2f4a4cd5f1637eba020358bfe5ecb5edcf2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_short-8-102ad2dea8d94528b402d980a45d53d4
@@ -0,0 +1 @@
+15
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 b/sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95
deleted file mode 100644
index c7930257dfef505fd996e1d6f22f2f35149990d0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95
+++ /dev/null
@@ -1 +0,0 @@
-7
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e b/sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e
deleted file mode 100644
index 7813681f5b41c028345ca62a2be376bae70b7f61..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e
+++ /dev/null
@@ -1 +0,0 @@
-5
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-9-d571e18b7d8ad44fef2e0b2424f34a0d b/sql/hive/src/test/resources/golden/udf_xpath_short-9-d571e18b7d8ad44fef2e0b2424f34a0d
new file mode 100644
index 0000000000000000000000000000000000000000..7f8f011eb73d6043d2e6db9d2c101195ae2801f2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_xpath_short-9-d571e18b7d8ad44fef2e0b2424f34a0d
@@ -0,0 +1 @@
+7
diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/udf_xpath_string-0-50131c0ba7b7a6b65c789a5a8497bada
similarity index 100%
rename from sql/hive/src/test/resources/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf
rename to sql/hive/src/test/resources/golden/udf_xpath_string-0-50131c0ba7b7a6b65c789a5a8497bada
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 b/sql/hive/src/test/resources/golden/udf_xpath_string-1-e315d11e9feb29177b5cb1e221c7cfa4
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4
rename to sql/hive/src/test/resources/golden/udf_xpath_string-1-e315d11e9feb29177b5cb1e221c7cfa4
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc b/sql/hive/src/test/resources/golden/udf_xpath_string-10-d87fb71039c9d2419d750a0721c5696f
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc
rename to sql/hive/src/test/resources/golden/udf_xpath_string-10-d87fb71039c9d2419d750a0721c5696f
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 b/sql/hive/src/test/resources/golden/udf_xpath_string-2-110b583cde6cd23c486d8223c444cbe9
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9
rename to sql/hive/src/test/resources/golden/udf_xpath_string-2-110b583cde6cd23c486d8223c444cbe9
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f b/sql/hive/src/test/resources/golden/udf_xpath_string-3-17e4d8122b93a1ebdba6c1d2cf9ce0c4
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f
rename to sql/hive/src/test/resources/golden/udf_xpath_string-3-17e4d8122b93a1ebdba6c1d2cf9ce0c4
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a b/sql/hive/src/test/resources/golden/udf_xpath_string-4-302630fe7dac2cc61fe7d36ead0f41ab
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a
rename to sql/hive/src/test/resources/golden/udf_xpath_string-4-302630fe7dac2cc61fe7d36ead0f41ab
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a b/sql/hive/src/test/resources/golden/udf_xpath_string-5-19357ba9cb87d3a5717543d2afdc96e2
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a
rename to sql/hive/src/test/resources/golden/udf_xpath_string-5-19357ba9cb87d3a5717543d2afdc96e2
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 b/sql/hive/src/test/resources/golden/udf_xpath_string-6-4837e470f745487fae4f498b3a2946bc
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14
rename to sql/hive/src/test/resources/golden/udf_xpath_string-6-4837e470f745487fae4f498b3a2946bc
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 b/sql/hive/src/test/resources/golden/udf_xpath_string-7-8e42951d002e3c4034b4a51928442706
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61
rename to sql/hive/src/test/resources/golden/udf_xpath_string-7-8e42951d002e3c4034b4a51928442706
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f b/sql/hive/src/test/resources/golden/udf_xpath_string-8-fdfa4e17d70608dcc634c9e1e8a8f288
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f
rename to sql/hive/src/test/resources/golden/udf_xpath_string-8-fdfa4e17d70608dcc634c9e1e8a8f288
diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 b/sql/hive/src/test/resources/golden/udf_xpath_string-9-84109613320bd05abccd1058044d62c3
similarity index 100%
rename from sql/hive/src/test/resources/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89
rename to sql/hive/src/test/resources/golden/udf_xpath_string-9-84109613320bd05abccd1058044d62c3
diff --git a/sql/hive/src/test/resources/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a b/sql/hive/src/test/resources/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a
deleted file mode 100644
index 599bf1880a83a648142915a564d5b8b29e790983..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a
+++ /dev/null
@@ -1 +0,0 @@
-stack(n, cols...) - turns k columns into n rows of size k/n each
diff --git a/sql/hive/src/test/resources/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 b/sql/hive/src/test/resources/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 b/sql/hive/src/test/resources/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 b/sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3
index e82f93fe65c5ecc4f66c376bf403637ead15459c..f00ecaf75c08420ed450b72014deb6ee0cb7e4e3 100644
--- a/sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3
+++ b/sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3
@@ -1,18 +1,18 @@
 # col_name            	data_type           	comment             
 	 	 
-a                   	string              	None                
+a                   	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Fri Jan 03 18:39:52 PST 2014	 
+CreateTime:         	Tue Oct 21 05:37:51 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/k1	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/k1	 
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1388803192          
+	transient_lastDdlTime	1413895071          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
@@ -24,4 +24,4 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	field.delim         	\u0001              
-	serialization.format	\u0001              
\ No newline at end of file
+	serialization.format	\u0001              
diff --git a/sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 b/sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3
index e82f93fe65c5ecc4f66c376bf403637ead15459c..f00ecaf75c08420ed450b72014deb6ee0cb7e4e3 100644
--- a/sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3
+++ b/sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3
@@ -1,18 +1,18 @@
 # col_name            	data_type           	comment             
 	 	 
-a                   	string              	None                
+a                   	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Fri Jan 03 18:39:52 PST 2014	 
+CreateTime:         	Tue Oct 21 05:37:51 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/k1	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/k1	 
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1388803192          
+	transient_lastDdlTime	1413895071          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
@@ -24,4 +24,4 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	field.delim         	\u0001              
-	serialization.format	\u0001              
\ No newline at end of file
+	serialization.format	\u0001              
diff --git a/sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 b/sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3
index 2be72c6e4e8c7de1d00f40ab3ee82053cacf6bb2..79e197569a51c4b17eb227bf910c38ff0e78c5fc 100644
--- a/sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3
+++ b/sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3
@@ -1,18 +1,18 @@
 # col_name            	data_type           	comment             
 	 	 
-a                   	string              	None                
+a                   	string              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Fri Jan 03 18:39:52 PST 2014	 
+CreateTime:         	Tue Oct 21 05:37:51 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/k1	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/k1	 
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1388803192          
+	transient_lastDdlTime	1413895071          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
@@ -24,4 +24,4 @@ Bucket Columns:     	[]
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
 	field.delim         	|                   
-	serialization.format	|                   
\ No newline at end of file
+	serialization.format	|                   
diff --git a/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52
+++ b/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 b/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856
index 1d5891034ddec9a4c5539f356d35fb8f1db40d1a..6489eb562dbf71c01d6d02df6f4b4fbe916c206c 100644
--- a/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856
+++ b/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856
@@ -1,3 +1,3 @@
 tst1	500
 tst2	500
-tst3	500
\ No newline at end of file
+tst3	500
diff --git a/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52
+++ b/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f b/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f
index 1ec0096b865a72d0e6f40ca6f61bbe1ca83b517f..c86307863b92d8a3cbe63e90e002bf245d2b98c7 100644
--- a/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f
+++ b/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f
@@ -1,3 +1,3 @@
 tst1	1
 tst2	1
-tst3	1
\ No newline at end of file
+tst3	1
diff --git a/sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 b/sql/hive/src/test/resources/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 b/sql/hive/src/test/resources/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 b/sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20
index 2209b11d742829679a225fe81cd460e4f6462bed..f085b06e588d432f96c686e44a420c2175613c22 100644
--- a/sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20
+++ b/sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20
@@ -997,4 +997,4 @@
 200	val_200
 200	val_200
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52
+++ b/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 b/sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398
index 07f67df79dcbda48c1beff97edc85fc560d9a1f2..5ba4e492aa0f46e3e9a6db7cb343ba7c66d8dc28 100644
--- a/sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398
+++ b/sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398
@@ -14,4 +14,4 @@ NULL	10
 406	1
 66	1
 98	1
-tst1	1
\ No newline at end of file
+tst1	1
diff --git a/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52
+++ b/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 b/sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0
index b00b55f6fa92d71e0710f6e2bca9383f456293ac..20cb88d87f6bf48b74ab4b1a37eafd1b61c132e7 100644
--- a/sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0
+++ b/sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0
@@ -14,4 +14,4 @@ NULL	20
 406	2
 66	2
 98	2
-tst1	1
\ No newline at end of file
+tst1	1
diff --git a/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382 b/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382
index 6f680072350ab1300a20d8e0187776a9a94523a9..eeb33ca418288543b06c91d4b8eccc5a5a379c0f 100644
--- a/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382
+++ b/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382
@@ -1 +1 @@
-12500
\ No newline at end of file
+12500
diff --git a/sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5
index a5a9e42fff2093e195451c5dbd3b929f5d3b3d44..4d199c29dd4db0381d29f41e3f5b624bfe6e4acb 100644
--- a/sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5
+++ b/sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5
@@ -307,4 +307,4 @@
 96	1
 97	1
 98	1
-tst1	1
\ No newline at end of file
+tst1	1
diff --git a/sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9
index be13b26eadc864f1091e710cf7e7171082d7db71..5b6f2e2d4f7736c284c39622645f1d6995a41d44 100644
--- a/sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9
+++ b/sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9
@@ -307,4 +307,4 @@
 96	val_96	1
 97	val_97	1
 98	val_98	1
-tst1	500	1
\ No newline at end of file
+tst1	500	1
diff --git a/sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 b/sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878
index 3dee790d7c2ca0a19aa7a6d75d283235d7741e55..11a5a2c531a3d7601d0e135ca1ff5d178adc1ff8 100644
--- a/sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878
+++ b/sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878
@@ -498,4 +498,4 @@
 97	val_97
 98	val_98
 98	val_98
-tst1	500
\ No newline at end of file
+tst1	500
diff --git a/sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 b/sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0
index e438a6405072339998c3fe671dc713bb23fa8e53..b95429f0c39e8453de515b3867771a2d58aa4dc3 100644
--- a/sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0
+++ b/sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0
@@ -498,4 +498,4 @@
 97	val_97	val_97
 98	val_98	val_98
 98	val_98	val_98
-tst1	500	500
\ No newline at end of file
+tst1	500	500
diff --git a/sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 b/sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878
index e4dfefc43e26abbb293138d3ef6718325d94bf39..f21cd3313298c84b9f1a26a660ab755659dadb38 100644
--- a/sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878
+++ b/sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878
@@ -307,4 +307,4 @@
 96	1
 97	2
 98	2
-tst1	1
\ No newline at end of file
+tst1	1
diff --git a/sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 b/sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0
index e438a6405072339998c3fe671dc713bb23fa8e53..b95429f0c39e8453de515b3867771a2d58aa4dc3 100644
--- a/sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0
+++ b/sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0
@@ -498,4 +498,4 @@
 97	val_97	val_97
 98	val_98	val_98
 98	val_98	val_98
-tst1	500	500
\ No newline at end of file
+tst1	500	500
diff --git a/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 b/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4
index e37d32abba426c06b752a5e53f48c595c84e9270..83b33d238dab9943201aaf267f701e8ea5fc9268 100644
--- a/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4
+++ b/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4
@@ -1 +1 @@
-1000
\ No newline at end of file
+1000
diff --git a/sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c b/sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c
index 385b8df6703a069a3771bd0c81e6280139db0d0f..f8539c4817e21054033a3c826beba6bada774320 100644
--- a/sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c
+++ b/sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c
@@ -20,4 +20,4 @@
 5	val_5	5	val_5
 5	val_5	5	val_5
 9	val_9	9	val_9
-tst1	500	tst1	500
\ No newline at end of file
+tst1	500	tst1	500
diff --git a/sql/hive/src/test/resources/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c b/sql/hive/src/test/resources/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 b/sql/hive/src/test/resources/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13
deleted file mode 100644
index 26a2aff52e9a96b343539f91cc873ce4f69569a5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13
+++ /dev/null
@@ -1,536 +0,0 @@
-NULL	2
-0	7
-001	2
-002	2
-004	1
-01	1
-013	1
-02	1
-021	2
-03	1
-032	5
-034	3
-051	1
-061	1
-062	1
-063	1
-064	1
-07	3
-071	1
-074	1
-08	1
-081	1
-082	2
-084	3
-09	3
-091	1
-094	1
-1	500
-10	2
-100	2
-102	1
-103	2
-104	7
-105	1
-11	2
-111	2
-113	5
-114	2
-116	1
-118	2
-119	3
-12	2
-120	2
-122	2
-123	2
-124	1
-125	2
-126	1
-128	3
-129	2
-131	2
-133	3
-134	5
-136	1
-137	2
-138	4
-14	1
-142	1
-143	2
-145	1
-146	2
-149	2
-15	4
-150	1
-152	2
-153	2
-155	1
-156	1
-157	1
-158	1
-160	1
-162	1
-163	1
-164	2
-165	2
-166	1
-167	3
-168	1
-169	4
-17	1
-170	1
-172	2
-174	2
-175	2
-176	2
-177	1
-178	1
-179	2
-18	2
-180	1
-181	2
-182	2
-183	1
-184	1
-186	1
-187	3
-189	1
-19	1
-190	1
-191	4
-192	2
-193	3
-194	2
-195	2
-196	1
-197	2
-199	3
-2	2
-20	2
-200	2
-201	1
-202	2
-203	3
-204	1
-205	2
-207	2
-208	3
-209	2
-21	2
-213	2
-214	1
-216	2
-217	2
-218	1
-219	2
-221	2
-222	2
-223	4
-224	2
-226	1
-228	1
-229	2
-230	5
-233	3
-234	1
-235	1
-237	2
-238	2
-239	2
-24	4
-241	1
-242	4
-243	2
-244	1
-247	1
-248	1
-249	1
-251	2
-252	2
-254	1
-255	2
-256	2
-257	1
-258	1
-26	2
-260	1
-261	1
-262	2
-263	2
-264	2
-265	2
-266	1
-27	3
-271	2
-272	4
-273	3
-274	2
-275	1
-277	4
-278	2
-28	2
-280	2
-281	2
-282	4
-283	3
-284	2
-285	1
-286	1
-287	1
-288	2
-289	1
-29	1
-291	2
-292	2
-293	1
-294	2
-296	1
-298	3
-30	2
-301	2
-302	3
-304	3
-305	1
-306	1
-307	2
-308	1
-309	2
-310	1
-311	5
-312	2
-314	2
-315	1
-316	3
-317	2
-318	3
-321	2
-322	4
-323	2
-325	2
-327	3
-33	2
-331	3
-332	3
-333	4
-335	1
-336	1
-338	1
-339	1
-34	2
-341	2
-342	2
-344	3
-345	1
-348	5
-35	4
-351	2
-353	4
-354	1
-356	1
-360	1
-361	1
-362	2
-364	3
-365	1
-366	1
-367	2
-368	1
-369	3
-37	2
-372	3
-373	2
-374	1
-375	1
-377	1
-378	1
-379	1
-38	2
-381	1
-382	3
-384	4
-386	1
-389	1
-391	3
-392	1
-393	2
-394	2
-395	2
-396	3
-397	2
-399	2
-4	2
-40	1
-400	1
-401	7
-402	1
-403	3
-404	4
-406	4
-407	1
-409	3
-41	1
-411	2
-412	1
-413	2
-414	4
-417	3
-418	1
-419	1
-42	4
-421	1
-422	2
-424	4
-427	1
-429	2
-43	2
-430	3
-431	5
-432	1
-435	1
-436	1
-437	1
-438	3
-439	2
-44	2
-442	1
-443	3
-444	2
-446	1
-448	1
-449	1
-45	1
-452	1
-453	1
-454	6
-455	1
-457	1
-458	2
-459	2
-46	1
-460	1
-461	2
-462	2
-463	3
-466	3
-467	1
-468	4
-469	5
-47	2
-470	1
-471	2
-472	2
-473	1
-475	1
-477	1
-478	2
-479	1
-48	2
-480	3
-481	1
-482	2
-483	4
-484	2
-485	1
-487	1
-489	4
-490	1
-491	2
-492	2
-493	2
-494	2
-495	1
-496	1
-497	1
-498	3
-5	6
-50	1
-501	1
-502	2
-503	1
-51	4
-513	1
-521	2
-523	2
-53	4
-532	1
-533	1
-534	1
-54	1
-541	1
-543	1
-551	1
-552	2
-554	1
-56	1
-561	2
-562	2
-563	1
-57	1
-571	2
-572	1
-573	1
-574	1
-58	3
-582	1
-584	1
-59	2
-591	2
-593	2
-594	1
-60	1
-603	1
-604	4
-611	1
-612	2
-613	3
-62	2
-621	1
-622	1
-631	1
-633	1
-634	1
-64	1
-641	2
-644	1
-65	1
-651	1
-652	2
-653	1
-66	2
-661	1
-662	1
-663	1
-664	3
-67	4
-671	2
-68	1
-681	1
-682	1
-683	1
-69	2
-691	1
-692	1
-693	3
-694	1
-70	4
-702	2
-703	2
-704	1
-71	1
-712	2
-713	2
-714	3
-72	3
-723	3
-724	1
-73	2
-731	2
-732	2
-734	1
-74	2
-742	1
-75	1
-751	1
-752	1
-754	1
-76	4
-761	3
-763	2
-764	1
-77	2
-771	1
-772	4
-773	1
-774	1
-78	2
-781	3
-782	1
-784	1
-79	2
-791	2
-793	2
-794	1
-8	2
-80	2
-802	3
-803	1
-81	2
-811	2
-812	1
-813	3
-814	1
-82	2
-821	3
-822	1
-83	2
-831	4
-832	2
-833	1
-834	3
-84	2
-842	1
-843	5
-844	1
-85	3
-851	1
-852	1
-854	2
-86	1
-861	1
-863	1
-864	4
-87	2
-871	1
-872	2
-873	1
-874	2
-882	2
-89	2
-892	3
-894	3
-9	2
-90	4
-902	2
-903	2
-904	3
-91	1
-911	3
-912	2
-914	1
-92	1
-921	2
-922	2
-924	2
-932	2
-933	1
-934	2
-941	2
-942	1
-944	1
-95	2
-954	2
-96	2
-961	4
-963	3
-964	5
-97	2
-971	2
-973	1
-974	1
-98	2
-981	1
-982	1
-983	1
-984	4
-991	3
-993	2
-record_0	1
-record_1	1
-record_2	1
-record_3	1
-record_4	1
-record_5	1
-record_6	1
-record_7	1
-record_8	1
-record_9	1
diff --git a/sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd b/sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd
index bf0f76662bd2c59fafbd4c2b100ea0187a8db07e..14ac2a0543eb758d70d10f4dff01a216016d009d 100644
--- a/sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd
+++ b/sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd
@@ -997,4 +997,4 @@
 98	val_98
 98	val_98
 98	val_98
-98	val_98
\ No newline at end of file
+98	val_98
diff --git a/sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530 b/sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530
index 199095f1f9848ec5e06a56edd3a1a9af24bf6c3f..61d7280758de940eafa630e0cc890bc5fbbc6372 100644
--- a/sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530
+++ b/sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530
@@ -5,4 +5,4 @@
 97	val_97
 97	val_97
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 b/sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9
index 293f3242974197b14f28ddc52939716f3f048dc2..0e14af56f8b0881a122fc18697935e5c7b9e4aba 100644
--- a/sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9
+++ b/sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9
@@ -17,4 +17,4 @@
 8	val_8
 8	val_8
 8	val_8
-9	val_9
\ No newline at end of file
+9	val_9
diff --git a/sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 b/sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9
index 87a971baed4288eee5e9ae4f18db047630f5d557..ba254c7ab023bb0f44c85e1b217019e358a7fe80 100644
--- a/sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9
+++ b/sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9
@@ -17,4 +17,4 @@
 5	val_5
 5	val_5
 5	val_5
-5	val_5
\ No newline at end of file
+5	val_5
diff --git a/sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 b/sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9
index a6f502e6c3e3af3b3bf7ee3c71ec094e287912c1..996f02bc6496ca905b2d190abed5405fd7609d4a 100644
--- a/sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9
+++ b/sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9
@@ -17,4 +17,4 @@
 5	val_5
 5	val_5
 5	val_5
-5	val_5
\ No newline at end of file
+5	val_5
diff --git a/sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 b/sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125
index dadf53962e7a107b3f21d2765775d49a975822bb..b1fb9cce36fa2c9bb8aaa7e4bb33d1adf3367ba1 100644
--- a/sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125
+++ b/sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125
@@ -3,4 +3,4 @@
 4	2
 5	6
 8	2
-9	2
\ No newline at end of file
+9	2
diff --git a/sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e b/sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e
index dadf53962e7a107b3f21d2765775d49a975822bb..b1fb9cce36fa2c9bb8aaa7e4bb33d1adf3367ba1 100644
--- a/sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e
+++ b/sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e
@@ -3,4 +3,4 @@
 4	2
 5	6
 8	2
-9	2
\ No newline at end of file
+9	2
diff --git a/sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5 b/sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5
index ede645acaf95b8d0d77a8b92ea44af581c192baa..48255af9043494a03611205e98a43fd57e898a2e 100644
--- a/sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5
+++ b/sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5
@@ -3,4 +3,4 @@
 4	2
 5	2
 8	2
-9	2
\ No newline at end of file
+9	2
diff --git a/sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045 b/sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045
index ede645acaf95b8d0d77a8b92ea44af581c192baa..48255af9043494a03611205e98a43fd57e898a2e 100644
--- a/sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045
+++ b/sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045
@@ -3,4 +3,4 @@
 4	2
 5	2
 8	2
-9	2
\ No newline at end of file
+9	2
diff --git a/sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb b/sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb
index dadf53962e7a107b3f21d2765775d49a975822bb..b1fb9cce36fa2c9bb8aaa7e4bb33d1adf3367ba1 100644
--- a/sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb
+++ b/sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb
@@ -3,4 +3,4 @@
 4	2
 5	6
 8	2
-9	2
\ No newline at end of file
+9	2
diff --git a/sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 b/sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760
index a71793e8c52a3fa197ae5a57aaa797ddecd92196..c7f78ecade3e8dafbaf75e0850f0f8d593399eb6 100644
--- a/sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760
+++ b/sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760
@@ -3,4 +3,4 @@ val_2	2
 val_4	2
 val_5	6
 val_8	2
-val_9	2
\ No newline at end of file
+val_9	2
diff --git a/sql/hive/src/test/resources/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e b/sql/hive/src/test/resources/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 b/sql/hive/src/test/resources/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b b/sql/hive/src/test/resources/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 b/sql/hive/src/test/resources/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5
deleted file mode 100644
index 9bf0de2f06c9a94259e3a120d5944a16b5be944c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5
+++ /dev/null
@@ -1,32 +0,0 @@
-0.0	0.0
-0.0	0.0
-0.0	0.0
-0.0	0.0
-0.0	0.0
-0.0	0.0
-0.0	0.0
-0.0	0.0
-0.0	0.0
-0.0	0.0
-0.0	0.0
-0.0	0.0
-2.0	2.0
-2.0	2.0
-4.0	4.0
-4.0	4.0
-5.0	5.0
-5.0	5.0
-5.0	5.0
-5.0	5.0
-5.0	5.0
-5.0	5.0
-5.0	5.0
-5.0	5.0
-5.0	5.0
-5.0	5.0
-5.0	5.0
-5.0	5.0
-8.0	8.0
-8.0	8.0
-9.0	9.0
-9.0	9.0
diff --git a/sql/hive/src/test/resources/golden/union32-2-8e39fef33c859ef83912d0fcda319218 b/sql/hive/src/test/resources/golden/union32-2-8e39fef33c859ef83912d0fcda319218
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 b/sql/hive/src/test/resources/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84
deleted file mode 100644
index 462dca3124c41899b6fcf936d85a068e480e7554..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84
+++ /dev/null
@@ -1,20 +0,0 @@
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-2.0
-2.0
-4.0
-4.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-8.0
-8.0
-9.0
-9.0
diff --git a/sql/hive/src/test/resources/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e b/sql/hive/src/test/resources/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union32-5-51c997d0a1103b60764bbb8316a38746 b/sql/hive/src/test/resources/golden/union32-5-51c997d0a1103b60764bbb8316a38746
deleted file mode 100644
index 2e662f39d15720882982a75470eb2cc97d5f25ed..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union32-5-51c997d0a1103b60764bbb8316a38746
+++ /dev/null
@@ -1,32 +0,0 @@
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-2.0
-2.0
-4.0
-4.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-8.0
-8.0
-9.0
-9.0
diff --git a/sql/hive/src/test/resources/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b b/sql/hive/src/test/resources/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union32-7-f936440d63f4e1027dda4de605660633 b/sql/hive/src/test/resources/golden/union32-7-f936440d63f4e1027dda4de605660633
deleted file mode 100644
index 2e662f39d15720882982a75470eb2cc97d5f25ed..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union32-7-f936440d63f4e1027dda4de605660633
+++ /dev/null
@@ -1,32 +0,0 @@
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-0.0
-2.0
-2.0
-4.0
-4.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-5.0
-8.0
-8.0
-9.0
-9.0
diff --git a/sql/hive/src/test/resources/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb b/sql/hive/src/test/resources/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union32-9-74fb695786df4c024288ae23ac8c00db b/sql/hive/src/test/resources/golden/union32-9-74fb695786df4c024288ae23ac8c00db
deleted file mode 100644
index 106f1874451ff8ee935748ec168004ebcc6b8f0c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union32-9-74fb695786df4c024288ae23ac8c00db
+++ /dev/null
@@ -1,32 +0,0 @@
-0.0	0
-0.0	0.0
-0.0	0.0
-0.0	0.0
-0.0	0
-0.0	0.0
-0.0	0.0
-0.0	0.0
-0.0	0
-0.0	0.0
-0.0	0.0
-0.0	0.0
-2.0	2.0
-2.0	2
-4.0	4
-4.0	4.0
-5.0	5
-5.0	5.0
-5.0	5.0
-5.0	5.0
-5.0	5
-5.0	5.0
-5.0	5.0
-5.0	5.0
-5.0	5
-5.0	5.0
-5.0	5.0
-5.0	5.0
-8.0	8.0
-8.0	8
-9.0	9
-9.0	9.0
diff --git a/sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 b/sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043
index d572335ff518528c3252901e20a7756789327b14..7369ee231754098de2fe4d468357a51981069131 100644
--- a/sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043
+++ b/sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043
@@ -27,4 +27,4 @@
 86	val_86
 98	val_98
 98	val_98
-98	val_98
\ No newline at end of file
+98	val_98
diff --git a/sql/hive/src/test/resources/golden/union34-4-101829a66cab2efd31dcb0d86e302956 b/sql/hive/src/test/resources/golden/union34-4-101829a66cab2efd31dcb0d86e302956
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c b/sql/hive/src/test/resources/golden/union34-4-70479e10c016e5ac448394dbadb32794
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c
rename to sql/hive/src/test/resources/golden/union34-4-70479e10c016e5ac448394dbadb32794
diff --git a/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833
+++ b/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 b/sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043
index d572335ff518528c3252901e20a7756789327b14..7369ee231754098de2fe4d468357a51981069131 100644
--- a/sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043
+++ b/sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043
@@ -27,4 +27,4 @@
 86	val_86
 98	val_98
 98	val_98
-98	val_98
\ No newline at end of file
+98	val_98
diff --git a/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b
+++ b/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52
+++ b/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 b/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856
index 948aca9180ba929a4cf3a71ea458b870740940ce..abc382d9b3571698f75f772d2937383dcdc3801f 100644
--- a/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856
+++ b/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856
@@ -1,2 +1,2 @@
 tst1	500
-tst2	500
\ No newline at end of file
+tst2	500
diff --git a/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52
+++ b/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 b/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292
index c0c7d9f5e8ef54a4ec2989c986bdbbe76a6eb2db..9daa3f92b46e0e180fde82a9c770bea27eff8568 100644
--- a/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292
+++ b/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292
@@ -1,2 +1,2 @@
 tst1	1
-tst2	1
\ No newline at end of file
+tst2	1
diff --git a/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52
+++ b/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 b/sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15
index b5e91e032212a05cf2ada19181057ca12bf529d9..6f79451ce69b5f9e62ff8142f8127d3ea6ba2f6e 100644
--- a/sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15
+++ b/sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15
@@ -23,4 +23,4 @@ NULL	val_484
 406	val_406
 66	val_66
 98	val_98
-tst1	500
\ No newline at end of file
+tst1	500
diff --git a/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52
+++ b/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 b/sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5
index 07f67df79dcbda48c1beff97edc85fc560d9a1f2..5ba4e492aa0f46e3e9a6db7cb343ba7c66d8dc28 100644
--- a/sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5
+++ b/sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5
@@ -14,4 +14,4 @@ NULL	10
 406	1
 66	1
 98	1
-tst1	1
\ No newline at end of file
+tst1	1
diff --git a/sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771 b/sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771
index 36179099029936daa04840847644e2cb594f351c..52982787dbb4aa64e18573438f95b7830665675f 100644
--- a/sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771
+++ b/sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771
@@ -1497,4 +1497,4 @@
 200	val_200
 97	val_97
 97	val_97
-97	val_97
\ No newline at end of file
+97	val_97
diff --git a/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 b/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1
index 37021f4a27201e7257a4ac7a992b0b4ebba61a01..3d86ec6498f3f2c00a73220120de77934b185d8d 100644
--- a/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1
+++ b/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1
@@ -1 +1 @@
-1500
\ No newline at end of file
+1500
diff --git a/sql/hive/src/test/resources/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c b/sql/hive/src/test/resources/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 b/sql/hive/src/test/resources/golden/union_date-4-d85fe746334b430941c5db3665e744d4
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01
rename to sql/hive/src/test/resources/golden/union_date-4-d85fe746334b430941c5db3665e744d4
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 b/sql/hive/src/test/resources/golden/union_date-5-82eebfded24cef08e0a881d1bcca02b1
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0
rename to sql/hive/src/test/resources/golden/union_date-5-82eebfded24cef08e0a881d1bcca02b1
diff --git a/sql/hive/src/test/resources/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d b/sql/hive/src/test/resources/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a b/sql/hive/src/test/resources/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a
deleted file mode 100644
index 468f774ef514637203c6235efdcfab68a9cb2917..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a
+++ /dev/null
@@ -1,10 +0,0 @@
-val_238
-NULL
-val_86
-NULL
-val_311
-NULL
-val_27
-NULL
-val_165
-NULL
diff --git a/sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 b/sql/hive/src/test/resources/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index c203753d8e42d61a2f5c52c9095e52cd740b1ef0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	None                
-values              	bigint              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:49:42 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069004          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 b/sql/hive/src/test/resources/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 b/sql/hive/src/test/resources/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 b/sql/hive/src/test/resources/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 b/sql/hive/src/test/resources/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index 94a711cdab117e21f8f6eaaefc3743c7df4906d1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-values              	bigint              	from deserializer   
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:50:08 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069036          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 b/sql/hive/src/test/resources/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0
+++ b/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ b/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7
index d9cfa9e952c3aba6a5d3ccb44a5f64a6cb71ef44..4c47424eb1160fcec8ab2818a8e5d6d1a287fbe4 100644
--- a/sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7
+++ b/sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7
@@ -1,19 +1,24 @@
 # col_name            	data_type           	comment             
 	 	 
-key                 	string              	from deserializer   
-values              	bigint              	from deserializer   
+key                 	string              	                    
+values              	bigint              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Fri Jan 03 19:05:51 PST 2014	 
+CreateTime:         	Tue Oct 21 05:59:44 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/outputtbl1	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/outputtbl1	 
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1388804758          
+	COLUMN_STATS_ACCURATE	false               
+	numFiles            	1                   
+	numRows             	-1                  
+	rawDataSize         	-1                  
+	totalSize           	184                 
+	transient_lastDdlTime	1413896391          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
@@ -24,4 +29,4 @@ Num Buckets:        	-1
 Bucket Columns:     	[]                  	 
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a
+++ b/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e b/sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e
index 2817d74afee13859008fd98fb4625d84554f4e03..8930b40bed84fa613963b6cecf5f09d92b745d77 100644
--- a/sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e
+++ b/sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e
@@ -15,4 +15,4 @@
 8	2
 8	2
 8	3
-8	3
\ No newline at end of file
+8	3
diff --git a/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557
+++ b/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a
+++ b/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027
+++ b/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc
+++ b/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785
+++ b/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 b/sql/hive/src/test/resources/golden/union_remove_11-9-94da21f150ed2c56046b80e46da8884d
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97
rename to sql/hive/src/test/resources/golden/union_remove_11-9-94da21f150ed2c56046b80e46da8884d
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c b/sql/hive/src/test/resources/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 b/sql/hive/src/test/resources/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index 57d5146698ee753642b9a120b573cdaebf4d4115..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-values              	bigint              	from deserializer   
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:50:41 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069061          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 b/sql/hive/src/test/resources/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 b/sql/hive/src/test/resources/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af b/sql/hive/src/test/resources/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index c914024c14c8e2cedf11a44ce4befa194f80863f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-values              	bigint              	from deserializer   
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:51:05 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069086          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb b/sql/hive/src/test/resources/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c b/sql/hive/src/test/resources/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 b/sql/hive/src/test/resources/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index 5a55428afbe1a243a1b1239028b5488e30569f3d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-values              	bigint              	from deserializer   
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:51:30 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069108          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d b/sql/hive/src/test/resources/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 b/sql/hive/src/test/resources/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 b/sql/hive/src/test/resources/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index a2441e330ffc61129c9461b1db4018eae4b8be9d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-values              	bigint              	from deserializer   
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:51:52 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069112          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47
deleted file mode 100644
index a58273a1b054e6b37d5e1ec1d01b95a6d7401cbf..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47
+++ /dev/null
@@ -1,2 +0,0 @@
-ds=1
-ds=2
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d b/sql/hive/src/test/resources/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef b/sql/hive/src/test/resources/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef b/sql/hive/src/test/resources/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 b/sql/hive/src/test/resources/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 b/sql/hive/src/test/resources/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index bec8202dcbcad07204aeb7268ddace609117cd82..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-values              	bigint              	from deserializer   
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:52:19 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069139          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47
deleted file mode 100644
index a58273a1b054e6b37d5e1ec1d01b95a6d7401cbf..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47
+++ /dev/null
@@ -1,2 +0,0 @@
-ds=1
-ds=2
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a b/sql/hive/src/test/resources/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 b/sql/hive/src/test/resources/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d b/sql/hive/src/test/resources/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index d0b37e7625a64ee4ab2e19cddb81ad8782237250..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-values              	bigint              	from deserializer   
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Tue Jan 14 14:38:18 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4075462935071533647/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1389739098          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47
deleted file mode 100644
index c21f4017362c10777bae22d6659f60f8977ee1fa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47
+++ /dev/null
@@ -1,2 +0,0 @@
-ds=1
-ds=2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 b/sql/hive/src/test/resources/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6
deleted file mode 100644
index 1ab5c482feca9a782dd7e29a144398d372f0aec4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6
+++ /dev/null
@@ -1,6 +0,0 @@
-1	1	1
-2	1	1
-3	1	1
-7	1	1
-8	1	1
-8	1	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a b/sql/hive/src/test/resources/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a
deleted file mode 100644
index 94ec062e2f4f356c0ec2c4d709dad4a6e08c1919..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a
+++ /dev/null
@@ -1,6 +0,0 @@
-1	2	2
-2	2	2
-3	2	2
-7	2	2
-8	2	2
-8	2	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b b/sql/hive/src/test/resources/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef b/sql/hive/src/test/resources/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad b/sql/hive/src/test/resources/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 b/sql/hive/src/test/resources/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index 971e095df7be8f50a4b90bc2ae680cc4ca30570b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,32 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	None                
-values              	bigint              	None                
-	 	 
-# Partition Information	 	 
-# col_name            	data_type           	comment             
-	 	 
-ds                  	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:52:47 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069167          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47
deleted file mode 100644
index 5690d94a6585748b0fb78620cb09b4856c4d61fc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47
+++ /dev/null
@@ -1,6 +0,0 @@
-ds=11
-ds=12
-ds=13
-ds=17
-ds=18
-ds=28
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 b/sql/hive/src/test/resources/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 b/sql/hive/src/test/resources/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b b/sql/hive/src/test/resources/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index 0d0d3a30af36c8a85f2864545eee24fecb4105b6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	None                
-values              	bigint              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:53:14 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069215          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 b/sql/hive/src/test/resources/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387
deleted file mode 100644
index 719e4a7bd4dbb135cb71f99a60f19106b6812867..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387
+++ /dev/null
@@ -1,10 +0,0 @@
-1	1
-2	1
-3	1
-7	1
-8	2
-1	1
-2	1
-3	1
-7	1
-8	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 b/sql/hive/src/test/resources/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 b/sql/hive/src/test/resources/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 b/sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387
deleted file mode 100644
index 361f2ab78eae3dc535d51f7418e185f48b9c7c1d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387
+++ /dev/null
@@ -1,2 +0,0 @@
-7	1
-7	1
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e b/sql/hive/src/test/resources/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 b/sql/hive/src/test/resources/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 b/sql/hive/src/test/resources/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 b/sql/hive/src/test/resources/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 b/sql/hive/src/test/resources/golden/union_remove_2-10-2309570010c3e679b884c100de57d002
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index d3c821fa221824deebc3a7de515547be1a980cac..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	None                
-values              	bigint              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:53:36 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069242          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e b/sql/hive/src/test/resources/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f b/sql/hive/src/test/resources/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 b/sql/hive/src/test/resources/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index 978049640929c9ecf5c1323d985589aa07b31b8e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-values              	bigint              	None                
-key                 	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:54:06 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069268          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e b/sql/hive/src/test/resources/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f b/sql/hive/src/test/resources/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa b/sql/hive/src/test/resources/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 b/sql/hive/src/test/resources/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index ce7e0438f9efffa936497ad452c3d5ddf5c69f03..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,26 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:54:32 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069293          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f b/sql/hive/src/test/resources/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 b/sql/hive/src/test/resources/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 b/sql/hive/src/test/resources/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 b/sql/hive/src/test/resources/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index c261ce48bdfb91c8f3f41caa37763c9268e91209..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,28 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	None                
-values              	bigint              	None                
-values2             	bigint              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:54:58 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069319          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 b/sql/hive/src/test/resources/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387
deleted file mode 100644
index d36ac6b8347927d883097e91e80a786d3b88959b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387
+++ /dev/null
@@ -1,10 +0,0 @@
-1	1	1
-2	1	1
-3	1	1
-7	1	1
-8	2	2
-1	1	1
-2	1	1
-3	1	1
-7	1	1
-8	2	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd b/sql/hive/src/test/resources/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af b/sql/hive/src/test/resources/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 b/sql/hive/src/test/resources/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 b/sql/hive/src/test/resources/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb b/sql/hive/src/test/resources/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 b/sql/hive/src/test/resources/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index 50297abd4e9a647b0b0f61456f9d34b7290828b7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	None                
-values              	bigint              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:55:20 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069345          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 b/sql/hive/src/test/resources/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 b/sql/hive/src/test/resources/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 b/sql/hive/src/test/resources/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index f3c452ebe92840b9a8d10f5139e9ba253922d093..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	double              	None                
-values              	bigint              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:55:49 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069371          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 b/sql/hive/src/test/resources/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e b/sql/hive/src/test/resources/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f b/sql/hive/src/test/resources/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0
+++ b/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ b/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7
index d46af80f0d1ae860ec3c375dfc891c2a577e8934..8cd7d42b20df5a8bba5df210e59cda4a257a484e 100644
--- a/sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7
+++ b/sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7
@@ -1,19 +1,24 @@
 # col_name            	data_type           	comment             
 	 	 
-key                 	string              	None                
-values              	bigint              	None                
+key                 	string              	                    
+values              	bigint              	                    
 	 	 
 # Detailed Table Information	 	 
 Database:           	default             	 
 Owner:              	marmbrus            	 
-CreateTime:         	Fri Jan 03 19:09:55 PST 2014	 
+CreateTime:         	Tue Oct 21 06:00:02 PDT 2014	 
 LastAccessTime:     	UNKNOWN             	 
 Protect Mode:       	None                	 
 Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/outputtbl1	 
+Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/outputtbl1	 
 Table Type:         	MANAGED_TABLE       	 
 Table Parameters:	 	 
-	transient_lastDdlTime	1388805002          
+	COLUMN_STATS_ACCURATE	false               
+	numFiles            	1                   
+	numRows             	-1                  
+	rawDataSize         	-1                  
+	totalSize           	72                  
+	transient_lastDdlTime	1413896409          
 	 	 
 # Storage Information	 	 
 SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
@@ -24,4 +29,4 @@ Num Buckets:        	-1
 Bucket Columns:     	[]                  	 
 Sort Columns:       	[]                  	 
 Storage Desc Params:	 	 
-	serialization.format	1                   
\ No newline at end of file
+	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a
+++ b/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e b/sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e
index 2817d74afee13859008fd98fb4625d84554f4e03..8930b40bed84fa613963b6cecf5f09d92b745d77 100644
--- a/sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e
+++ b/sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e
@@ -15,4 +15,4 @@
 8	2
 8	2
 8	3
-8	3
\ No newline at end of file
+8	3
diff --git a/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557
+++ b/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab
+++ b/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785
+++ b/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/union_remove_3-8-94da21f150ed2c56046b80e46da8884d
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b
rename to sql/hive/src/test/resources/golden/union_remove_3-8-94da21f150ed2c56046b80e46da8884d
diff --git a/sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 b/sql/hive/src/test/resources/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 b/sql/hive/src/test/resources/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index 67faa5b13d984462aa430378194621137d602675..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	None                
-values              	bigint              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:56:16 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069398          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead b/sql/hive/src/test/resources/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f b/sql/hive/src/test/resources/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 b/sql/hive/src/test/resources/golden/union_remove_5-11-2309570010c3e679b884c100de57d002
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index 19aeb0b6028594a0557b5db203cd129790a9545d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	None                
-values              	bigint              	None                
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:56:42 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069430          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe	 
-InputFormat:        	org.apache.hadoop.mapred.TextInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 b/sql/hive/src/test/resources/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0
+++ b/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ b/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a
+++ b/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e b/sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e
index 8e7ee8a2b47bbd539510e6780239c7e83907bdb9..b4ea93dc7dd895ae39b80a423ef314f870fe3f3a 100644
--- a/sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e
+++ b/sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e
@@ -7,4 +7,4 @@
 7	1
 7	1
 8	2
-8	2
\ No newline at end of file
+8	2
diff --git a/sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 b/sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38
index 8e7ee8a2b47bbd539510e6780239c7e83907bdb9..b4ea93dc7dd895ae39b80a423ef314f870fe3f3a 100644
--- a/sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38
+++ b/sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38
@@ -7,4 +7,4 @@
 7	1
 7	1
 8	2
-8	2
\ No newline at end of file
+8	2
diff --git a/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557
+++ b/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523
+++ b/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab
+++ b/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785
index c227083464fb9af8955c90d2924774ee50abb547..573541ac9702dd3969c9bc859d2b91ec1f7e6e56 100644
--- a/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785
+++ b/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785
@@ -1 +1 @@
-0
\ No newline at end of file
+0
diff --git a/sql/hive/src/test/resources/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 b/sql/hive/src/test/resources/golden/union_remove_6-9-94da21f150ed2c56046b80e46da8884d
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8
rename to sql/hive/src/test/resources/golden/union_remove_6-9-94da21f150ed2c56046b80e46da8884d
diff --git a/sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 b/sql/hive/src/test/resources/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index 1c63b3851aad212b22dc9c8a683cd83b96a6dd2a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-values              	bigint              	from deserializer   
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:57:18 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069463          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca b/sql/hive/src/test/resources/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 b/sql/hive/src/test/resources/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 b/sql/hive/src/test/resources/golden/union_remove_8-10-2309570010c3e679b884c100de57d002
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index f718ab2990cdd8dec584bd59948d530aa9f33671..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-values              	bigint              	from deserializer   
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:57:48 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069495          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee b/sql/hive/src/test/resources/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f b/sql/hive/src/test/resources/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 b/sql/hive/src/test/resources/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c b/sql/hive/src/test/resources/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7
deleted file mode 100644
index 65a91a6027545978b04e0db9c0fd62b8ad0af757..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7
+++ /dev/null
@@ -1,27 +0,0 @@
-# col_name            	data_type           	comment             
-	 	 
-key                 	string              	from deserializer   
-values              	bigint              	from deserializer   
-	 	 
-# Detailed Table Information	 	 
-Database:           	default             	 
-Owner:              	marmbrus            	 
-CreateTime:         	Mon Feb 10 13:58:19 PST 2014	 
-LastAccessTime:     	UNKNOWN             	 
-Protect Mode:       	None                	 
-Retention:          	0                   	 
-Location:           	file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1	 
-Table Type:         	MANAGED_TABLE       	 
-Table Parameters:	 	 
-	transient_lastDdlTime	1392069518          
-	 	 
-# Storage Information	 	 
-SerDe Library:      	org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe	 
-InputFormat:        	org.apache.hadoop.hive.ql.io.RCFileInputFormat	 
-OutputFormat:       	org.apache.hadoop.hive.ql.io.RCFileOutputFormat	 
-Compressed:         	No                  	 
-Num Buckets:        	-1                  	 
-Bucket Columns:     	[]                  	 
-Sort Columns:       	[]                  	 
-Storage Desc Params:	 	 
-	serialization.format	1                   
diff --git a/sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 b/sql/hive/src/test/resources/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 b/sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6
index 42d07ed10bad4a0ab53dfba289f8f8d52607d610..760e62cce0bba58355393c48dde47d544558b9e0 100644
--- a/sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6
+++ b/sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6
@@ -497,4 +497,4 @@
 97
 97
 98
-98
\ No newline at end of file
+98
diff --git a/sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 b/sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769
index 2f23db6a2ff4a4b357c32e26cab6b879864566fd..3bafdf359f6faff5d7b8a19dd535b6a166c7a9ce 100644
--- a/sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769
+++ b/sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769
@@ -997,4 +997,4 @@
 98
 98
 98
-98
\ No newline at end of file
+98
diff --git a/sql/hive/src/test/resources/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d b/sql/hive/src/test/resources/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_view-1-c790d4344144460224b0f02be7e137a8 b/sql/hive/src/test/resources/golden/union_view-1-c790d4344144460224b0f02be7e137a8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 b/sql/hive/src/test/resources/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 b/sql/hive/src/test/resources/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e b/sql/hive/src/test/resources/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 b/sql/hive/src/test/resources/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae b/sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 b/sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 b/sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8
deleted file mode 100644
index 573541ac9702dd3969c9bc859d2b91ec1f7e6e56..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8
+++ /dev/null
@@ -1 +0,0 @@
-0
diff --git a/sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da b/sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da
index 08661a3d7f794eb201a5852be95e71bbf03d003a..6fc243e718c1b1190a5e553258f179580ec15018 100644
--- a/sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da
+++ b/sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da
@@ -2,4 +2,4 @@ val_0	0	3
 val_10	10	1
 val_100	200	2
 val_103	206	2
-val_104	208	2
\ No newline at end of file
+val_104	208	2
diff --git a/sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f b/sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f
index 08661a3d7f794eb201a5852be95e71bbf03d003a..6fc243e718c1b1190a5e553258f179580ec15018 100644
--- a/sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f
+++ b/sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f
@@ -2,4 +2,4 @@ val_0	0	3
 val_10	10	1
 val_100	200	2
 val_103	206	2
-val_104	208	2
\ No newline at end of file
+val_104	208	2
diff --git a/sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 b/sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519
index f0c744ca37d9560f7969c5053a97f0aa666cdf90..0052fe0c959c5cbeec13454c153f26e4c19e1cee 100644
--- a/sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519
+++ b/sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519
@@ -2,4 +2,4 @@ val_98	196	2
 val_97	194	2
 val_96	96	1
 val_95	190	2
-val_92	92	1
\ No newline at end of file
+val_92	92	1
diff --git a/sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f b/sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f
index f0c744ca37d9560f7969c5053a97f0aa666cdf90..0052fe0c959c5cbeec13454c153f26e4c19e1cee 100644
--- a/sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f
+++ b/sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f
@@ -2,4 +2,4 @@ val_98	196	2
 val_97	194	2
 val_96	96	1
 val_95	190	2
-val_92	92	1
\ No newline at end of file
+val_92	92	1
diff --git a/sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce b/sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce
index 35e7795b9c2a75ac860a4e212fa442c4da0a01ed..fa55ed96ee0b9ad0fb8589534607ce13c2c09c9f 100644
--- a/sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce
+++ b/sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce
@@ -1,3 +1,3 @@
 1	abc	1	abc
 2	abc 	2	abc 
-3	 abc	3	 abc
\ No newline at end of file
+3	 abc	3	 abc
diff --git a/sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c b/sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c
index 35e7795b9c2a75ac860a4e212fa442c4da0a01ed..fa55ed96ee0b9ad0fb8589534607ce13c2c09c9f 100644
--- a/sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c
+++ b/sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c
@@ -1,3 +1,3 @@
 1	abc	1	abc
 2	abc 	2	abc 
-3	 abc	3	 abc
\ No newline at end of file
+3	 abc	3	 abc
diff --git a/sql/hive/src/test/resources/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef b/sql/hive/src/test/resources/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 b/sql/hive/src/test/resources/golden/varchar_join1-6-ab4392aa5ff499ec43229425ff23e22f
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9
rename to sql/hive/src/test/resources/golden/varchar_join1-6-ab4392aa5ff499ec43229425ff23e22f
diff --git a/sql/hive/src/test/resources/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 b/sql/hive/src/test/resources/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 b/sql/hive/src/test/resources/golden/varchar_join1-7-a01639290aaceb4b85aa6e44319f6386
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37
rename to sql/hive/src/test/resources/golden/varchar_join1-7-a01639290aaceb4b85aa6e44319f6386
diff --git a/sql/hive/src/test/resources/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 b/sql/hive/src/test/resources/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 b/sql/hive/src/test/resources/golden/varchar_join1-8-c05d7b534b51cecdc2ba2de4ce57ba37
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715
rename to sql/hive/src/test/resources/golden/varchar_join1-8-c05d7b534b51cecdc2ba2de4ce57ba37
diff --git a/sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 b/sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82
index 35e7795b9c2a75ac860a4e212fa442c4da0a01ed..fa55ed96ee0b9ad0fb8589534607ce13c2c09c9f 100644
--- a/sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82
+++ b/sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82
@@ -1,3 +1,3 @@
 1	abc	1	abc
 2	abc 	2	abc 
-3	 abc	3	 abc
\ No newline at end of file
+3	 abc	3	 abc
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc b/sql/hive/src/test/resources/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 b/sql/hive/src/test/resources/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 b/sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628
deleted file mode 100644
index 05066db2967561124a619c6c0b6193ab597af956..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628
+++ /dev/null
@@ -1 +0,0 @@
-c1                  	array<varchar(20)>  	None                
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 b/sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238
deleted file mode 100644
index 05cd8fccf276cdfb9226a6c228afee85d32997d7..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238
+++ /dev/null
@@ -1 +0,0 @@
-["val_0","val_0"]
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f b/sql/hive/src/test/resources/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 b/sql/hive/src/test/resources/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 b/sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763
deleted file mode 100644
index 1ec6828af9fd26535777911843214eabf5654972..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763
+++ /dev/null
@@ -1 +0,0 @@
-c1                  	map<int,varchar(20)>	None                
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 b/sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215
deleted file mode 100644
index 8f12062de4e2719f65d3e95e74226728c5f909e5..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215
+++ /dev/null
@@ -1 +0,0 @@
-{0:"val_0"}
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 b/sql/hive/src/test/resources/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc b/sql/hive/src/test/resources/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 b/sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1
deleted file mode 100644
index 030d9d6ec0429f77366e41ddec968274df4bd739..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1
+++ /dev/null
@@ -1 +0,0 @@
-c1                  	struct<a:int,b:varchar(20),c:string>	None                
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 b/sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377
deleted file mode 100644
index f350c1112ac2c0e48c989024a777c14c3a22f28b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377
+++ /dev/null
@@ -1 +0,0 @@
-{"a":0,"b":"val_0","c":"val_0"}
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb b/sql/hive/src/test/resources/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 b/sql/hive/src/test/resources/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad b/sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad
deleted file mode 100644
index 030d9d6ec0429f77366e41ddec968274df4bd739..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad
+++ /dev/null
@@ -1 +0,0 @@
-c1                  	struct<a:int,b:varchar(20),c:string>	None                
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 b/sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8
deleted file mode 100644
index f350c1112ac2c0e48c989024a777c14c3a22f28b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8
+++ /dev/null
@@ -1 +0,0 @@
-{"a":0,"b":"val_0","c":"val_0"}
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 b/sql/hive/src/test/resources/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 b/sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730
deleted file mode 100644
index 030d9d6ec0429f77366e41ddec968274df4bd739..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730
+++ /dev/null
@@ -1 +0,0 @@
-c1                  	struct<a:int,b:varchar(20),c:string>	None                
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb b/sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb
deleted file mode 100644
index f350c1112ac2c0e48c989024a777c14c3a22f28b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb
+++ /dev/null
@@ -1 +0,0 @@
-{"a":0,"b":"val_0","c":"val_0"}
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc b/sql/hive/src/test/resources/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 b/sql/hive/src/test/resources/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb b/sql/hive/src/test/resources/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 b/sql/hive/src/test/resources/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 b/sql/hive/src/test/resources/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 b/sql/hive/src/test/resources/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 b/sql/hive/src/test/resources/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 b/sql/hive/src/test/resources/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 b/sql/hive/src/test/resources/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b b/sql/hive/src/test/resources/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 b/sql/hive/src/test/resources/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 b/sql/hive/src/test/resources/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 b/sql/hive/src/test/resources/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 b/sql/hive/src/test/resources/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d b/sql/hive/src/test/resources/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 b/sql/hive/src/test/resources/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e b/sql/hive/src/test/resources/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 b/sql/hive/src/test/resources/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e b/sql/hive/src/test/resources/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e
deleted file mode 100644
index 3f02a574d3dd59b4b982142214c8609da9aa47d6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e
+++ /dev/null
@@ -1,5 +0,0 @@
-474	val_475
-62	val_63
-468	val_469
-272	val_273
-448	val_449
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 b/sql/hive/src/test/resources/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13
deleted file mode 100644
index 1e2a6a6562290e18ac1c53b375080d14f4292cdc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13
+++ /dev/null
@@ -1,5 +0,0 @@
-val_0	3
-val_1	2
-val_10	1
-val_100	2
-val_101	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 b/sql/hive/src/test/resources/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 b/sql/hive/src/test/resources/golden/varchar_serde-16-516202183287d734d35d8c7788d22652
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a b/sql/hive/src/test/resources/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d b/sql/hive/src/test/resources/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d
deleted file mode 100644
index 3f02a574d3dd59b4b982142214c8609da9aa47d6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d
+++ /dev/null
@@ -1,5 +0,0 @@
-474	val_475
-62	val_63
-468	val_469
-272	val_273
-448	val_449
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad b/sql/hive/src/test/resources/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad
deleted file mode 100644
index 1e2a6a6562290e18ac1c53b375080d14f4292cdc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad
+++ /dev/null
@@ -1,5 +0,0 @@
-val_0	3
-val_1	2
-val_10	1
-val_100	2
-val_101	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e b/sql/hive/src/test/resources/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 b/sql/hive/src/test/resources/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b b/sql/hive/src/test/resources/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 b/sql/hive/src/test/resources/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 b/sql/hive/src/test/resources/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70
deleted file mode 100644
index 3f02a574d3dd59b4b982142214c8609da9aa47d6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70
+++ /dev/null
@@ -1,5 +0,0 @@
-474	val_475
-62	val_63
-468	val_469
-272	val_273
-448	val_449
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 b/sql/hive/src/test/resources/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575
deleted file mode 100644
index 1e2a6a6562290e18ac1c53b375080d14f4292cdc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575
+++ /dev/null
@@ -1,5 +0,0 @@
-val_0	3
-val_1	2
-val_10	1
-val_100	2
-val_101	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc b/sql/hive/src/test/resources/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f b/sql/hive/src/test/resources/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 b/sql/hive/src/test/resources/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e b/sql/hive/src/test/resources/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e
deleted file mode 100644
index 3f02a574d3dd59b4b982142214c8609da9aa47d6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e
+++ /dev/null
@@ -1,5 +0,0 @@
-474	val_475
-62	val_63
-468	val_469
-272	val_273
-448	val_449
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c b/sql/hive/src/test/resources/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c
deleted file mode 100644
index 1e2a6a6562290e18ac1c53b375080d14f4292cdc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c
+++ /dev/null
@@ -1,5 +0,0 @@
-val_0	3
-val_1	2
-val_10	1
-val_100	2
-val_101	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 b/sql/hive/src/test/resources/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a b/sql/hive/src/test/resources/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 b/sql/hive/src/test/resources/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb b/sql/hive/src/test/resources/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 b/sql/hive/src/test/resources/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364
deleted file mode 100644
index 3f02a574d3dd59b4b982142214c8609da9aa47d6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364
+++ /dev/null
@@ -1,5 +0,0 @@
-474	val_475
-62	val_63
-468	val_469
-272	val_273
-448	val_449
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 b/sql/hive/src/test/resources/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9
deleted file mode 100644
index 1e2a6a6562290e18ac1c53b375080d14f4292cdc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9
+++ /dev/null
@@ -1,5 +0,0 @@
-val_0	3
-val_1	2
-val_10	1
-val_100	2
-val_101	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 b/sql/hive/src/test/resources/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d b/sql/hive/src/test/resources/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e b/sql/hive/src/test/resources/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 b/sql/hive/src/test/resources/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f b/sql/hive/src/test/resources/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f b/sql/hive/src/test/resources/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba b/sql/hive/src/test/resources/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba b/sql/hive/src/test/resources/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 b/sql/hive/src/test/resources/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 b/sql/hive/src/test/resources/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 b/sql/hive/src/test/resources/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244
deleted file mode 100644
index 3f02a574d3dd59b4b982142214c8609da9aa47d6..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244
+++ /dev/null
@@ -1,5 +0,0 @@
-474	val_475
-62	val_63
-468	val_469
-272	val_273
-448	val_449
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 b/sql/hive/src/test/resources/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601
deleted file mode 100644
index 1e2a6a6562290e18ac1c53b375080d14f4292cdc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601
+++ /dev/null
@@ -1,5 +0,0 @@
-val_0	3
-val_1	2
-val_10	1
-val_100	2
-val_101	2
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 b/sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715
index 173eb955a485ae8508b330b26c60255f3e7a1547..52bb2199e16a3bdfd09512f5ecf7a063b8edda75 100644
--- a/sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715
+++ b/sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715
@@ -1,4 +1,4 @@
 1	abc
 1	abc
 2	abc 
-3	 abc
\ No newline at end of file
+3	 abc
diff --git a/sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 b/sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9
index 173eb955a485ae8508b330b26c60255f3e7a1547..52bb2199e16a3bdfd09512f5ecf7a063b8edda75 100644
--- a/sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9
+++ b/sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9
@@ -1,4 +1,4 @@
 1	abc
 1	abc
 2	abc 
-3	 abc
\ No newline at end of file
+3	 abc
diff --git a/sql/hive/src/test/resources/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 b/sql/hive/src/test/resources/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/varchar_union1-6-f338f341c5f86d0a44cabfb4f7bddc3b
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b
rename to sql/hive/src/test/resources/golden/varchar_union1-6-f338f341c5f86d0a44cabfb4f7bddc3b
diff --git a/sql/hive/src/test/resources/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 b/sql/hive/src/test/resources/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc b/sql/hive/src/test/resources/golden/varchar_union1-7-ea0d1fbae997b50dc34f7610480bbe29
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc
rename to sql/hive/src/test/resources/golden/varchar_union1-7-ea0d1fbae997b50dc34f7610480bbe29
diff --git a/sql/hive/src/test/resources/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 b/sql/hive/src/test/resources/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e b/sql/hive/src/test/resources/golden/varchar_union1-8-f3be9a2498927d692356c2cf871d25bf
similarity index 100%
rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e
rename to sql/hive/src/test/resources/golden/varchar_union1-8-f3be9a2498927d692356c2cf871d25bf
diff --git a/sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 b/sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110
index 173eb955a485ae8508b330b26c60255f3e7a1547..52bb2199e16a3bdfd09512f5ecf7a063b8edda75 100644
--- a/sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110
+++ b/sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110
@@ -1,4 +1,4 @@
 1	abc
 1	abc
 2	abc 
-3	 abc
\ No newline at end of file
+3	 abc
diff --git a/sql/hive/src/test/resources/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a b/sql/hive/src/test/resources/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 b/sql/hive/src/test/resources/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 b/sql/hive/src/test/resources/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-11-dc95343d3e57846485dd543476391376 b/sql/hive/src/test/resources/golden/view-11-dc95343d3e57846485dd543476391376
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-12-371764e1cae31ea0518c03060528d239 b/sql/hive/src/test/resources/golden/view-12-371764e1cae31ea0518c03060528d239
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-13-2abce88008f8a19164758ee821aaa8a6 b/sql/hive/src/test/resources/golden/view-13-2abce88008f8a19164758ee821aaa8a6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-14-deb504f4f70fd7db975950c3c47959ee b/sql/hive/src/test/resources/golden/view-14-deb504f4f70fd7db975950c3c47959ee
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b b/sql/hive/src/test/resources/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-2-9c529f486fa81a032bfe1253808fca8 b/sql/hive/src/test/resources/golden/view-2-9c529f486fa81a032bfe1253808fca8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-3-89c80c0e90409d5e304775c9f420915a b/sql/hive/src/test/resources/golden/view-3-89c80c0e90409d5e304775c9f420915a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-4-4a64d1a623ca71e515796787dbd0f904 b/sql/hive/src/test/resources/golden/view-4-4a64d1a623ca71e515796787dbd0f904
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b b/sql/hive/src/test/resources/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-6-47b5043f03a84695b6784682b4402ac8 b/sql/hive/src/test/resources/golden/view-6-47b5043f03a84695b6784682b4402ac8
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 b/sql/hive/src/test/resources/golden/view-7-8b1bbdadfd1e11af1b56064196164e58
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 b/sql/hive/src/test/resources/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 b/sql/hive/src/test/resources/golden/view-9-66c68babac10ae0f645fe8334c5a42d4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf b/sql/hive/src/test/resources/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb b/sql/hive/src/test/resources/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 b/sql/hive/src/test/resources/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de b/sql/hive/src/test/resources/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 b/sql/hive/src/test/resources/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 b/sql/hive/src/test/resources/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a b/sql/hive/src/test/resources/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 b/sql/hive/src/test/resources/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb b/sql/hive/src/test/resources/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 b/sql/hive/src/test/resources/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 b/sql/hive/src/test/resources/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 b/sql/hive/src/test/resources/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe b/sql/hive/src/test/resources/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d b/sql/hive/src/test/resources/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d
deleted file mode 100644
index ea00577174e436d4871b3e5c095742f11d665d99..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d
+++ /dev/null
@@ -1,500 +0,0 @@
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	238	0
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	86	12
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	311	22
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	27	34
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	165	44
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	409	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	255	68
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	278	80
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	98	92
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	484	102
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	265	114
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	193	126
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	401	138
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	150	150
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	273	162
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	224	174
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	369	186
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	66	198
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	128	208
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	213	220
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	146	232
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	406	244
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	429	256
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	374	268
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	152	280
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	469	292
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	145	304
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	495	316
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	37	328
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	327	338
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	281	350
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	277	362
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	209	374
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	15	386
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	82	396
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	403	406
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	166	418
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	417	430
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	430	442
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	252	454
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	292	466
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	219	478
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	287	490
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	153	502
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	193	514
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	338	526
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	446	538
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	459	550
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	394	562
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	237	574
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	482	586
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	174	598
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	413	610
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	494	622
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	207	634
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	199	646
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	466	658
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	208	670
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	174	682
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	399	694
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	396	706
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	247	718
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	417	730
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	489	742
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	162	754
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	377	766
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	397	778
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	309	790
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	365	802
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	266	814
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	439	826
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	342	838
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	367	850
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	325	862
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	167	874
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	195	886
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	475	898
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	17	910
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	113	920
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	155	932
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	203	944
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	339	956
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	0	968
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	455	976
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	128	988
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	311	1000
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	316	1012
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	57	1024
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	302	1034
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	205	1046
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	149	1058
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	438	1070
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	345	1082
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	129	1094
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	170	1106
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	20	1118
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	489	1128
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	157	1140
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	378	1152
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	221	1164
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	92	1176
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	111	1186
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	47	1198
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	72	1208
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	4	1218
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	280	1226
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	35	1238
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	427	1248
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	277	1260
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	208	1272
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	356	1284
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	399	1296
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	169	1308
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	382	1320
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	498	1332
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	125	1344
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	386	1356
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	437	1368
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	469	1380
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	192	1392
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	286	1404
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	187	1416
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	176	1428
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	54	1440
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	459	1450
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	51	1462
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	138	1472
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	103	1484
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	239	1496
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	213	1508
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	216	1520
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	430	1532
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	278	1544
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	176	1556
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	289	1568
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	221	1580
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	65	1592
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	318	1602
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	332	1614
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	311	1626
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	275	1638
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	137	1650
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	241	1662
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	83	1674
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	333	1684
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	180	1696
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	284	1708
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	12	1720
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	230	1730
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	181	1742
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	67	1754
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	260	1764
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	404	1776
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	384	1788
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	489	1800
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	353	1812
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	373	1824
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	272	1836
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	138	1848
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	217	1860
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	84	1872
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	348	1882
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	466	1894
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	58	1906
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	8	1916
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	411	1924
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	230	1936
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	208	1948
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	348	1960
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	24	1972
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	463	1982
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	431	1994
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	179	2006
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	172	2018
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	42	2030
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	129	2040
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	158	2052
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	119	2064
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	496	2076
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	0	2088
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	322	2096
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	197	2108
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	468	2120
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	393	2132
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	454	2144
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	100	2156
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	298	2168
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	199	2180
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	191	2192
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	418	2204
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	96	2216
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	26	2226
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	165	2236
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	327	2248
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	230	2260
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	205	2272
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	120	2284
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	131	2296
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	51	2308
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	404	2318
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	43	2330
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	436	2340
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	156	2352
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	469	2364
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	468	2376
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	308	2388
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	95	2400
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	196	2410
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	288	2422
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	481	2434
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	457	2446
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	98	2458
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	282	2468
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	197	2480
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	187	2492
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	318	2504
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	318	2516
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	409	2528
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	470	2540
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	137	2552
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	369	2564
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	316	2576
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	169	2588
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	413	2600
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	85	2612
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	77	2622
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	0	2632
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	490	2640
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	87	2652
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	364	2662
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	179	2674
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	118	2686
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	134	2698
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	395	2710
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	282	2722
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	138	2734
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	238	2746
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	419	2758
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	15	2770
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	118	2780
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	72	2792
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	90	2802
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	307	2812
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	19	2824
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	435	2834
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	10	2846
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	277	2856
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	273	2868
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	306	2880
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	224	2892
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	309	2904
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	389	2916
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	327	2928
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	242	2940
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	369	2952
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	392	2964
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	272	2976
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	331	2988
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	401	3000
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	242	3012
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	452	3024
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	177	3036
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	226	3048
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	5	3060
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	497	3068
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	402	3080
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	396	3092
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	317	3104
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	395	3116
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	58	3128
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	35	3138
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	336	3148
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	95	3160
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	11	3170
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	168	3180
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	34	3192
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	229	3202
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	233	3214
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	143	3226
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	472	3238
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	322	3250
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	498	3262
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	160	3274
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	195	3286
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	42	3298
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	321	3308
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	430	3320
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	119	3332
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	489	3344
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	458	3356
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	78	3368
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	76	3378
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	41	3388
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	223	3398
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	492	3410
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	149	3422
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	449	3434
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	218	3446
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	228	3458
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	138	3470
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	453	3482
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	30	3494
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	209	3504
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	64	3516
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	468	3526
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	76	3538
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	74	3548
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	342	3558
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	69	3570
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	230	3580
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	33	3592
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	368	3602
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	103	3614
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	296	3626
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	113	3638
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	216	3650
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	367	3662
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	344	3674
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	167	3686
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	274	3698
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	219	3710
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	239	3722
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	485	3734
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	116	3746
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	223	3758
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	256	3770
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	263	3782
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	70	3794
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	487	3804
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	480	3816
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	401	3828
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	288	3840
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	191	3852
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	5	3864
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	244	3872
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	438	3884
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	128	3896
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	467	3908
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	432	3920
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	202	3932
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	316	3944
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	229	3956
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	469	3968
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	463	3980
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	280	3992
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	2	4004
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	35	4012
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	283	4022
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	331	4034
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	235	4046
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	80	4058
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	44	4068
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	193	4078
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	321	4090
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	335	4102
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	104	4114
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	466	4126
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	366	4138
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	175	4150
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	403	4162
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	483	4174
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	53	4186
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	105	4196
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	257	4208
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	406	4220
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	409	4232
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	190	4244
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	406	4256
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	401	4268
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	114	4280
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	258	4292
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	90	4304
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	203	4314
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	262	4326
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	348	4338
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	424	4350
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	12	4362
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	396	4372
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	201	4384
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	217	4396
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	164	4408
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	431	4420
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	454	4432
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	478	4444
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	298	4456
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	125	4468
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	431	4480
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	164	4492
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	424	4504
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	187	4516
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	382	4528
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	5	4540
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	70	4548
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	397	4558
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	480	4570
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	291	4582
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	24	4594
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	351	4604
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	255	4616
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	104	4628
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	70	4640
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	163	4650
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	438	4662
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	119	4674
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	414	4686
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	200	4698
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	491	4710
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	237	4722
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	439	4734
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	360	4746
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	248	4758
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	479	4770
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	305	4782
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	417	4794
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	199	4806
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	444	4818
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	120	4830
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	429	4842
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	169	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	443	4866
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	323	4878
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	325	4890
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	277	4902
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	230	4914
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	478	4926
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	178	4938
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	468	4950
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	310	4962
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	317	4974
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	333	4986
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	493	4998
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	460	5010
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	207	5022
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	249	5034
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	265	5046
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	480	5058
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	83	5070
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	136	5080
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	353	5092
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	172	5104
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	214	5116
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	462	5128
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	233	5140
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	406	5152
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	133	5164
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	175	5176
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	189	5188
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	454	5200
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	375	5212
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	401	5224
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	421	5236
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	407	5248
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	384	5260
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	256	5272
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	26	5284
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	134	5294
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	67	5306
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	384	5316
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	379	5328
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	18	5340
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	462	5350
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	492	5362
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	100	5374
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	298	5386
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	9	5398
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	341	5406
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	498	5418
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	146	5430
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	458	5442
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	362	5454
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	186	5466
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	285	5478
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	348	5490
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	167	5502
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	18	5514
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	273	5524
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	183	5536
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	281	5548
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	344	5560
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	97	5572
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	469	5582
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	315	5594
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	84	5606
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	28	5616
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	37	5626
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	448	5636
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	152	5648
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	348	5660
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	307	5672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	194	5684
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	414	5696
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	477	5708
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	222	5720
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	126	5732
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	90	5744
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	169	5754
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	403	5766
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	400	5778
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	200	5790
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	97	5802
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 b/sql/hive/src/test/resources/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945
deleted file mode 100644
index df07a9da29f01512947648bed0e2a4f71e8798dc..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945
+++ /dev/null
@@ -1,309 +0,0 @@
-0	3
-2	1
-4	1
-5	3
-8	1
-9	1
-10	1
-11	1
-12	2
-15	2
-17	1
-18	2
-19	1
-20	1
-24	2
-26	2
-27	1
-28	1
-30	1
-33	1
-34	1
-35	3
-37	2
-41	1
-42	2
-43	1
-44	1
-47	1
-51	2
-53	1
-54	1
-57	1
-58	2
-64	1
-65	1
-66	1
-67	2
-69	1
-70	3
-72	2
-74	1
-76	2
-77	1
-78	1
-80	1
-82	1
-83	2
-84	2
-85	1
-86	1
-87	1
-90	3
-92	1
-95	2
-96	1
-97	2
-98	2
-100	2
-103	2
-104	2
-105	1
-111	1
-113	2
-114	1
-116	1
-118	2
-119	3
-120	2
-125	2
-126	1
-128	3
-129	2
-131	1
-133	1
-134	2
-136	1
-137	2
-138	4
-143	1
-145	1
-146	2
-149	2
-150	1
-152	2
-153	1
-155	1
-156	1
-157	1
-158	1
-160	1
-162	1
-163	1
-164	2
-165	2
-166	1
-167	3
-168	1
-169	4
-170	1
-172	2
-174	2
-175	2
-176	2
-177	1
-178	1
-179	2
-180	1
-181	1
-183	1
-186	1
-187	3
-189	1
-190	1
-191	2
-192	1
-193	3
-194	1
-195	2
-196	1
-197	2
-199	3
-200	2
-201	1
-202	1
-203	2
-205	2
-207	2
-208	3
-209	2
-213	2
-214	1
-216	2
-217	2
-218	1
-219	2
-221	2
-222	1
-223	2
-224	2
-226	1
-228	1
-229	2
-230	5
-233	2
-235	1
-237	2
-238	2
-239	2
-241	1
-242	2
-244	1
-247	1
-248	1
-249	1
-252	1
-255	2
-256	2
-257	1
-258	1
-260	1
-262	1
-263	1
-265	2
-266	1
-272	2
-273	3
-274	1
-275	1
-277	4
-278	2
-280	2
-281	2
-282	2
-283	1
-284	1
-285	1
-286	1
-287	1
-288	2
-289	1
-291	1
-292	1
-296	1
-298	3
-302	1
-305	1
-306	1
-307	2
-308	1
-309	2
-310	1
-311	3
-315	1
-316	3
-317	2
-318	3
-321	2
-322	2
-323	1
-325	2
-327	3
-331	2
-332	1
-333	2
-335	1
-336	1
-338	1
-339	1
-341	1
-342	2
-344	2
-345	1
-348	5
-351	1
-353	2
-356	1
-360	1
-362	1
-364	1
-365	1
-366	1
-367	2
-368	1
-369	3
-373	1
-374	1
-375	1
-377	1
-378	1
-379	1
-382	2
-384	3
-386	1
-389	1
-392	1
-393	1
-394	1
-395	2
-396	3
-397	2
-399	2
-400	1
-401	5
-402	1
-403	3
-404	2
-406	4
-407	1
-409	3
-411	1
-413	2
-414	2
-417	3
-418	1
-419	1
-421	1
-424	2
-427	1
-429	2
-430	3
-431	3
-432	1
-435	1
-436	1
-437	1
-438	3
-439	2
-443	1
-444	1
-446	1
-448	1
-449	1
-452	1
-453	1
-454	3
-455	1
-457	1
-458	2
-459	2
-460	1
-462	2
-463	2
-466	3
-467	1
-468	4
-469	5
-470	1
-472	1
-475	1
-477	1
-478	2
-479	1
-480	3
-481	1
-482	1
-483	1
-484	1
-485	1
-487	1
-489	4
-490	1
-491	1
-492	2
-493	1
-494	1
-495	1
-496	1
-497	1
-498	3
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 b/sql/hive/src/test/resources/golden/virtual_column-10-2915b222a58bc994246591e536d388b4
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 b/sql/hive/src/test/resources/golden/virtual_column-2-1536b365fe0a94b30a62364996529966
deleted file mode 100644
index 2a0cdc268347a1829f1ecc1dd29870f822908835..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/virtual_column-2-1536b365fe0a94b30a62364996529966
+++ /dev/null
@@ -1,309 +0,0 @@
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	0	[968,2632,2088]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	2	[4004]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	4	[1218]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	5	[4540,3864,3060]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	8	[1916]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	9	[5398]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	10	[2846]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	11	[3170]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	12	[4362,1720]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	15	[386,2770]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	17	[910]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	18	[5514,5340]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	19	[2824]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	20	[1118]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	24	[1972,4594]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	26	[5284,2226]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	27	[34]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	28	[5616]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	30	[3494]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	33	[3592]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	34	[3192]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	35	[1238,4012,3138]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	37	[328,5626]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	41	[3388]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	42	[3298,2030]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	43	[2330]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	44	[4068]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	47	[1198]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	51	[1462,2308]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	53	[4186]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	54	[1440]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	57	[1024]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	58	[3128,1906]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	64	[3516]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	65	[1592]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	66	[198]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	67	[5306,1754]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	69	[3570]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	70	[3794,4640,4548]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	72	[2792,1208]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	74	[3548]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	76	[3538,3378]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	77	[2622]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	78	[3368]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	80	[4058]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	82	[396]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	83	[5070,1674]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	84	[5606,1872]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	85	[2612]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	86	[12]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	87	[2652]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	90	[2802,5744,4304]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	92	[1176]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	95	[2400,3160]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	96	[2216]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	97	[5572,5802]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	98	[2458,92]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	100	[5374,2156]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	103	[3614,1484]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	104	[4628,4114]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	105	[4196]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	111	[1186]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	113	[920,3638]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	114	[4280]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	116	[3746]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	118	[2780,2686]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	119	[2064,4674,3332]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	120	[2284,4830]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	125	[4468,1344]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	126	[5732]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	128	[3896,988,208]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	129	[2040,1094]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	131	[2296]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	133	[5164]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	134	[2698,5294]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	136	[5080]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	137	[2552,1650]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	138	[2734,1848,1472,3470]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	143	[3226]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	145	[304]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	146	[232,5430]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	149	[3422,1058]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	150	[150]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	152	[280,5648]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	153	[502]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	155	[932]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	156	[2352]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	157	[1140]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	158	[2052]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	160	[3274]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	162	[754]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	163	[4650]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	164	[4492,4408]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	165	[2236,44]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	166	[418]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	167	[5502,874,3686]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	168	[3180]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	169	[5754,1308,2588,4854]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	170	[1106]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	172	[2018,5104]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	174	[682,598]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	175	[5176,4150]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	176	[1428,1556]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	177	[3036]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	178	[4938]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	179	[2674,2006]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	180	[1696]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	181	[1742]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	183	[5536]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	186	[5466]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	187	[2492,1416,4516]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	189	[5188]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	190	[4244]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	191	[3852,2192]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	192	[1392]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	193	[514,126,4078]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	194	[5684]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	195	[3286,886]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	196	[2410]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	197	[2480,2108]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	199	[646,2180,4806]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	200	[4698,5790]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	201	[4384]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	202	[3932]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	203	[4314,944]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	205	[1046,2272]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	207	[5022,634]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	208	[670,1948,1272]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	209	[3504,374]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	213	[220,1508]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	214	[5116]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	216	[1520,3650]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	217	[4396,1860]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	218	[3446]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	219	[478,3710]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	221	[1580,1164]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	222	[5720]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	223	[3398,3758]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	224	[174,2892]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	226	[3048]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	228	[3458]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	229	[3202,3956]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	230	[1730,1936,4914,2260,3580]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	233	[5140,3214]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	235	[4046]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	237	[4722,574]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	238	[0,2746]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	239	[1496,3722]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	241	[1662]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	242	[2940,3012]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	244	[3872]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	247	[718]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	248	[4758]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	249	[5034]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	252	[454]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	255	[68,4616]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	256	[5272,3770]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	257	[4208]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	258	[4292]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	260	[1764]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	262	[4326]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	263	[3782]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	265	[114,5046]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	266	[814]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	272	[1836,2976]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	273	[2868,5524,162]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	274	[3698]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	275	[1638]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	277	[4902,1260,2856,362]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	278	[1544,80]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	280	[3992,1226]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	281	[350,5548]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	282	[2468,2722]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	283	[4022]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	284	[1708]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	285	[5478]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	286	[1404]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	287	[490]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	288	[2422,3840]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	289	[1568]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	291	[4582]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	292	[466]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	296	[3626]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	298	[5386,4456,2168]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	302	[1034]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	305	[4782]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	306	[2880]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	307	[2812,5672]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	308	[2388]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	309	[790,2904]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	310	[4962]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	311	[1000,1626,22]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	315	[5594]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	316	[3944,2576,1012]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	317	[3104,4974]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	318	[2504,1602,2516]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	321	[4090,3308]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	322	[3250,2096]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	323	[4878]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	325	[4890,862]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	327	[2928,338,2248]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	331	[2988,4034]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	332	[1614]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	333	[1684,4986]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	335	[4102]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	336	[3148]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	338	[526]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	339	[956]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	341	[5406]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	342	[3558,838]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	344	[3674,5560]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	345	[1082]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	348	[5660,1882,1960,4338,5490]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	351	[4604]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	353	[1812,5092]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	356	[1284]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	360	[4746]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	362	[5454]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	364	[2662]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	365	[802]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	366	[4138]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	367	[850,3662]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	368	[3602]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	369	[186,2564,2952]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	373	[1824]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	374	[268]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	375	[5212]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	377	[766]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	378	[1152]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	379	[5328]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	382	[1320,4528]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	384	[5316,5260,1788]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	386	[1356]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	389	[2916]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	392	[2964]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	393	[2132]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	394	[562]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	395	[3116,2710]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	396	[4372,706,3092]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	397	[4558,778]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	399	[694,1296]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	400	[5778]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	401	[138,3000,3828,4268,5224]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	402	[3080]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	403	[5766,4162,406]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	404	[1776,2318]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	406	[244,4220,4256,5152]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	407	[5248]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	409	[4232,56,2528]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	411	[1924]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	413	[610,2600]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	414	[5696,4686]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	417	[730,4794,430]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	418	[2204]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	419	[2758]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	421	[5236]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	424	[4350,4504]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	427	[1248]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	429	[4842,256]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	430	[442,1532,3320]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	431	[4420,1994,4480]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	432	[3920]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	435	[2834]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	436	[2340]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	437	[1368]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	438	[3884,4662,1070]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	439	[4734,826]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	443	[4866]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	444	[4818]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	446	[538]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	448	[5636]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	449	[3434]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	452	[3024]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	453	[3482]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	454	[2144,4432,5200]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	455	[976]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	457	[2446]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	458	[3356,5442]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	459	[550,1450]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	460	[5010]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	462	[5350,5128]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	463	[1982,3980]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	466	[658,1894,4126]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	467	[3908]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	468	[3526,4950,2120,2376]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	469	[292,3968,1380,5582,2364]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	470	[2540]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	472	[3238]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	475	[898]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	477	[5708]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	478	[4444,4926]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	479	[4770]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	480	[4570,5058,3816]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	481	[2434]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	482	[586]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	483	[4174]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	484	[102]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	485	[3734]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	487	[3804]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	489	[1128,1800,3344,742]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	490	[2640]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	491	[4710]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	492	[5362,3410]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	493	[4998]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	494	[622]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	495	[316]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	496	[2076]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	497	[3068]
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt	498	[5418,3262,1332]
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/virtual_column-3-c66776673c986b59b27e704664935988 b/sql/hive/src/test/resources/golden/virtual_column-3-c66776673c986b59b27e704664935988
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 b/sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6
deleted file mode 100644
index 94754d8bd4c82e27d0b4289bfc98109002cd17e1..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6
+++ /dev/null
@@ -1 +0,0 @@
-97	val_97
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 b/sql/hive/src/test/resources/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 b/sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7
deleted file mode 100644
index c227083464fb9af8955c90d2924774ee50abb547..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7
+++ /dev/null
@@ -1 +0,0 @@
-0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 b/sql/hive/src/test/resources/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 b/sql/hive/src/test/resources/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7
deleted file mode 100644
index e6902f60ff486f5b5fbac3f13b03fd70148b0dd0..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7
+++ /dev/null
@@ -1,500 +0,0 @@
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	0	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	0	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	0	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	2	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	4	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	5	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	5	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	5	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	8	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	9	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	10	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	11	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	12	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	12	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	15	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	15	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	17	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	18	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	18	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	19	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	20	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	24	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	24	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	26	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	26	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	27	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	28	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	30	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	33	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	34	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	35	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	35	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	35	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	37	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	37	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	41	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	42	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	42	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	43	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	44	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	47	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	51	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	51	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	53	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	54	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	57	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	58	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	58	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	64	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	65	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	66	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	67	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	67	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	69	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	70	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	70	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	70	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	72	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	72	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	74	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	76	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	76	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	77	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	78	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	80	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	82	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	83	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	83	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	84	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	84	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	85	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	86	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	87	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	90	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	90	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	90	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	92	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	95	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	95	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	96	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	97	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	97	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	98	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	98	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	100	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	100	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	103	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	103	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	104	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	104	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	105	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	111	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	113	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	113	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	114	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	116	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	118	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	118	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	119	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	119	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	119	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	120	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	120	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	125	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	125	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	126	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	128	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	128	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	128	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	129	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	129	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	131	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	133	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	134	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	134	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	136	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	137	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	137	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	138	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	138	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	138	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	138	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	143	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	145	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	146	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	146	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	149	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	149	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	150	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	152	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	152	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	153	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	155	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	156	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	157	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	158	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	160	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	162	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	163	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	164	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	164	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	165	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	165	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	166	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	167	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	167	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	167	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	168	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	169	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	169	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	169	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	169	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	170	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	172	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	172	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	174	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	174	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	175	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	175	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	176	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	176	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	177	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	178	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	179	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	179	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	180	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	181	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	183	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	186	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	187	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	187	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	187	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	189	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	190	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	191	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	191	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	192	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	193	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	193	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	193	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	194	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	195	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	195	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	196	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	197	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	197	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	199	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	199	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	199	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	200	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	200	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	201	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	202	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	203	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	203	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	205	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	205	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	207	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	207	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	208	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	208	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	208	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	209	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	209	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	213	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	213	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	214	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	216	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	216	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	217	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	217	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	218	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	219	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	219	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	221	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	221	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	222	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	223	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	223	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	224	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	224	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	226	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	228	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	229	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	229	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	230	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	230	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	230	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	230	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	230	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	233	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	233	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	235	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	237	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	237	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	238	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	238	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	239	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	239	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	241	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	242	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	242	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	244	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	247	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	248	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	249	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	252	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	255	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	255	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	256	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	256	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	257	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	258	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	260	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	262	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	263	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	265	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	265	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	266	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	272	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	272	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	273	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	273	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	273	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	274	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	275	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	277	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	277	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	277	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	277	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	278	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	278	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	280	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	280	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	281	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	281	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	282	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	282	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	283	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	284	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	285	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	286	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	287	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	288	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	288	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	289	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	291	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	292	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	296	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	298	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	298	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	298	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	302	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	305	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	306	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	307	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	307	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	308	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	309	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	309	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	310	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	311	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	311	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	311	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	315	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	316	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	316	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	316	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	317	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	317	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	318	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	318	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	318	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	321	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	321	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	322	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	322	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	323	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	325	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	325	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	327	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	327	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	327	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	331	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	331	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	332	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	333	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	333	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	335	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	336	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	338	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	339	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	341	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	342	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	342	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	344	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	344	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	345	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	348	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	348	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	348	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	348	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	348	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	351	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	353	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	353	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	356	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	360	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	362	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	364	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	365	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	366	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	367	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	367	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	368	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	369	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	369	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	369	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	373	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	374	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	375	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	377	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	378	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	379	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	382	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	382	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	384	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	384	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	384	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	386	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	389	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	392	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	393	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	394	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	395	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	395	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	396	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	396	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	396	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	397	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	397	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	399	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	399	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	400	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	401	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	401	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	401	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	401	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	401	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	402	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	403	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	403	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	403	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	404	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	404	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	406	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	406	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	406	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	406	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	407	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	409	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	409	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	409	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	411	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	413	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	413	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	414	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	414	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	417	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	417	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	417	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	418	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	419	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	421	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	424	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	424	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	427	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	429	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	429	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	430	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	430	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	430	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	431	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	431	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	431	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	432	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	435	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	436	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	437	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	438	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	438	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	438	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	439	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	439	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	443	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	444	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	446	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	448	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	449	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	452	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	453	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	454	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	454	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	454	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	455	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	457	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	458	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	458	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	459	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	459	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	460	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	462	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	462	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	463	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	463	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	466	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	466	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	466	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	467	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	468	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	468	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	468	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	468	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	469	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	469	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	469	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	469	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	469	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	470	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	472	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	475	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	477	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	478	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	478	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	479	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	480	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	480	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	480	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	481	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	482	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	483	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	484	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	485	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	487	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	489	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	489	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	489	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	489	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	490	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	491	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	492	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	492	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	493	3672
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	494	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	495	56
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	496	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	497	2449
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	498	1249
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	498	4854
-file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0	498	2449
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 b/sql/hive/src/test/resources/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5
deleted file mode 100644
index e69de29bb2d1d6434b8b29ae775ad8c2e48c5391..0000000000000000000000000000000000000000
diff --git a/sql/hive/src/test/resources/log4j.properties b/sql/hive/src/test/resources/log4j.properties
index 9fdb526d945e0ce66192a53c24033d26600aa417..5bc08062d30eb6b761b138809e823344fa739f65 100644
--- a/sql/hive/src/test/resources/log4j.properties
+++ b/sql/hive/src/test/resources/log4j.properties
@@ -42,6 +42,12 @@ log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF
 log4j.additivity.org.apache.hadoop.hive.metastore.RetryingHMSHandler=false
 log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=OFF
 
+log4j.additivity.hive.log=false
+log4j.logger.hive.log=OFF
+
+log4j.additivity.parquet.hadoop.ParquetRecordReader=false
+log4j.logger.parquet.hadoop.ParquetRecordReader=OFF
+
 log4j.additivity.hive.ql.metadata.Hive=false
 log4j.logger.hive.ql.metadata.Hive=OFF
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q
new file mode 100644
index 0000000000000000000000000000000000000000..c640ca148b70b925e689e0a14e54e4483aa55d3d
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q
@@ -0,0 +1,43 @@
+
+select
+   cdouble
+  ,Round(cdouble, 2)
+  ,Floor(cdouble)
+  ,Ceil(cdouble)
+  ,Rand(98007) as rnd
+  ,Exp(ln(cdouble))
+  ,Ln(cdouble)  
+  ,Ln(cfloat)
+  ,Log10(cdouble)
+  -- Use log2 as a representative function to test all input types.
+  ,Log2(cdouble)
+  ,Log2(cfloat)
+  ,Log2(cbigint)
+  ,Log2(cint)
+  ,Log2(csmallint)
+  ,Log2(ctinyint)
+  ,Log(2.0, cdouble)
+  ,Pow(log2(cdouble), 2.0)  
+  ,Power(log2(cdouble), 2.0)
+  ,Sqrt(cdouble)
+  ,Sqrt(cbigint)
+  ,Bin(cbigint)
+  ,Hex(cdouble)
+  ,Conv(cbigint, 10, 16)
+  ,Abs(cdouble)
+  ,Abs(ctinyint)
+  ,Pmod(cint, 3)
+  ,Sin(cdouble)
+  ,Asin(cdouble)
+  ,Cos(cdouble)
+  ,ACos(cdouble)
+  ,Atan(cdouble)
+  ,Degrees(cdouble)
+  ,Radians(cdouble)
+  ,Positive(cdouble)
+  ,Positive(cbigint)
+  ,Negative(cdouble)
+  ,Sign(cdouble)
+  ,Sign(cbigint)
+from alltypesorc order by rnd limit 400;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv
new file mode 100644
index 0000000000000000000000000000000000000000..51f231008f6d24921cbbda54347bf3b6705e7adc
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv
@@ -0,0 +1 @@
+SET hive.vectorized.execution.enabled = false;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv
new file mode 100644
index 0000000000000000000000000000000000000000..18e02dc854baf11e12b1262087ec614c3a9e9e45
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv
@@ -0,0 +1 @@
+SET hive.vectorized.execution.enabled = true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q
index 4881757a4613a7bace9c015e3770a52b7650b4fe..4193315d30043e8ba9a39fb7b026d7a52126d028 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q
@@ -1,9 +1,9 @@
 set hive.exec.concatenate.check.index=true;
 create table src_rc_concatenate_test(key int, value string) stored as rcfile;
 
-load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_concatenate_test;
-load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_concatenate_test;
-load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_concatenate_test;
+load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_concatenate_test;
+load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_concatenate_test;
+load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_concatenate_test;
 
 show table extended like `src_rc_concatenate_test`;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q
index 5f9d5ef9ca09e879b0b65a58c91126ef3e17a59c..8cbb25cfa97251edf9787147e61482277d2a6806 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q
@@ -2,7 +2,7 @@
 create table if not exists alter_part_invalidspec(key string, value string ) partitioned by (year string, month string) stored as textfile ;
 
 -- Load data
-load data local inpath '../data/files/T1.txt' overwrite into table alter_part_invalidspec partition (year='1996', month='10');
-load data local inpath '../data/files/T1.txt' overwrite into table alter_part_invalidspec partition (year='1996', month='12');
+load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_invalidspec partition (year='1996', month='10');
+load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_invalidspec partition (year='1996', month='12');
 
 alter table alter_part_invalidspec partition (year='1997') enable no_drop;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q
index 92af30b6b5a1508e58b8410baaf3bf07665a8d33..3c0ff02b1ac1ddd19fe43de37aa327b48db410eb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q
@@ -2,8 +2,8 @@
 create table if not exists alter_part_nodrop_part(key string, value string ) partitioned by (year string, month string) stored as textfile ;
 
 -- Load data
-load data local inpath '../data/files/T1.txt' overwrite into table alter_part_nodrop_part partition (year='1996', month='10');
-load data local inpath '../data/files/T1.txt' overwrite into table alter_part_nodrop_part partition (year='1996', month='12');
+load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_part partition (year='1996', month='10');
+load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_part partition (year='1996', month='12');
 
 alter table alter_part_nodrop_part partition (year='1996') enable no_drop;
 alter table alter_part_nodrop_part drop partition (year='1996');
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q
index 135411fd320effcf280b123ca34caff747f2b6e2..f2135b1aa02e461b2f698ffc863498954d410a70 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q
@@ -2,8 +2,8 @@
 create table if not exists alter_part_nodrop_table(key string, value string ) partitioned by (year string, month string) stored as textfile ;
 
 -- Load data
-load data local inpath '../data/files/T1.txt' overwrite into table alter_part_nodrop_table partition (year='1996', month='10');
-load data local inpath '../data/files/T1.txt' overwrite into table alter_part_nodrop_table partition (year='1996', month='12');
+load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_table partition (year='1996', month='10');
+load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_table partition (year='1996', month='12');
 
 alter table alter_part_nodrop_table partition (year='1996') enable no_drop;
 drop table alter_part_nodrop_table;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q
index 899145deaf8c56778ed26e5df0a2114eeeeec2c7..7376d8bfe4a74aa35aaf98bb594d1d1a8ba88f2a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q
@@ -2,8 +2,8 @@
 create table if not exists alter_part_offline (key string, value string ) partitioned by (year string, month string) stored as textfile ;
 
 -- Load data
-load data local inpath '../data/files/T1.txt' overwrite into table alter_part_offline partition (year='1996', month='10');
-load data local inpath '../data/files/T1.txt' overwrite into table alter_part_offline partition (year='1996', month='12');
+load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_offline partition (year='1996', month='10');
+load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_offline partition (year='1996', month='12');
 
 alter table alter_part_offline partition (year='1996') disable offline;
 select * from alter_part_offline where year = '1996';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q
index 26ba2878903fe8aaa35972c72da271f209c35163..be971f18498693491d0699f7793a39e21b1ed684 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q
@@ -1,5 +1,5 @@
 create table alter_rename_partition_src ( col1 string ) stored as textfile ;
-load data local inpath '../data/files/test.dat' overwrite into table alter_rename_partition_src ;
+load data local inpath '../../data/files/test.dat' overwrite into table alter_rename_partition_src ;
 create table alter_rename_partition ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile;
 insert overwrite table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') select col1 from alter_rename_partition_src ;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q
index 6e51c2f7629c9c3e49963ae7f8c2aa0352e02b23..4babdda2dbe2ac894502a50bf7f7f9c42855a29e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q
@@ -1,5 +1,5 @@
 create table alter_rename_partition_src ( col1 string ) stored as textfile ;
-load data local inpath '../data/files/test.dat' overwrite into table alter_rename_partition_src ;
+load data local inpath '../../data/files/test.dat' overwrite into table alter_rename_partition_src ;
 create table alter_rename_partition ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile;
 insert overwrite table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') select col1 from alter_rename_partition_src ;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q
index 2d4ce0b9f6051787f94b9ad1af5dc639b00e7369..3af807ef61217253045468ff4c18414ce16969be 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q
@@ -1,5 +1,5 @@
 create table alter_rename_partition_src ( col1 string ) stored as textfile ;
-load data local inpath '../data/files/test.dat' overwrite into table alter_rename_partition_src ;
+load data local inpath '../../data/files/test.dat' overwrite into table alter_rename_partition_src ;
 create table alter_rename_partition ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile;
 insert overwrite table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') select col1 from alter_rename_partition_src ;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q
index fdf20f850e8f836b710d1e17a573d361780e4ce6..9e8bcbd1bbf78ec62ce41b5ad033c70f96cae4ee 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q
@@ -1,2 +1,3 @@
+set hive.support.quoted.identifiers=none;
 -- TOK_TABLE_OR_COL
 explain select * from (select `.*` from (select * from src) a join (select * from src1) b on (a.key = b.key)) t;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q
index de59bc579a0fe9e0240fe80c37b2ef6091ce9939..33d4aed3cd9ad54d395c18d5eaee060cbf7279e7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q
@@ -1,2 +1,3 @@
+set hive.support.quoted.identifiers=none;
 -- DOT
 explain select * from (select a.`[kv].*`, b.`[kv].*` from (select * from src) a join (select * from src1) b on (a.key = b.key)) t;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q
new file mode 100644
index 0000000000000000000000000000000000000000..78a97019f192e698ab6669c0fa51feb5b3b8c9dd
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q
@@ -0,0 +1 @@
+analyze table nonexistent compute statistics;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q
index bea25391628a6e5b94b1183dd52e5c54ea5cdcff..130b37b5c9d525bd7cda23ad1b1ea13b1dc371c7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q
@@ -14,5 +14,5 @@ create table tstsrcpart like srcpart;
 -- to be thrown during the LOAD step. This former behavior is tested
 -- in clientpositive/archive_corrupt.q
 
-load data local inpath '../data/files/archive_corrupt.rc' overwrite into table tstsrcpart partition (ds='2008-04-08', hr='11');
+load data local inpath '../../data/files/archive_corrupt.rc' overwrite into table tstsrcpart partition (ds='2008-04-08', hr='11');
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q
new file mode 100644
index 0000000000000000000000000000000000000000..a1709dae5f5b84149de46b9af0ddc9e31f902e64
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q
@@ -0,0 +1,7 @@
+set hive.security.authorization.enabled=true;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+
+-- running a sql query to initialize the authorization - not needed in real HS2 mode
+show tables;
+
+add jar ${system:maven.local.repository}/org/apache/hive/hcatalog/hive-hcatalog-core/${system:hive.version}/hive-hcatalog-core-${system:hive.version}.jar;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q
new file mode 100644
index 0000000000000000000000000000000000000000..8abdd2b3cde8b85bc248368aaa5939a458a41840
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q
@@ -0,0 +1,10 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+set user.name=user1;
+-- check add partition without insert privilege
+create table tpart(i int, j int) partitioned by (k string);         
+
+set user.name=user2;
+alter table tpart add partition (k = 'abc');
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q
new file mode 100644
index 0000000000000000000000000000000000000000..f716262e23bbbe38037e218bc43a577aeb0f944d
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q
@@ -0,0 +1,11 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=user1;
+
+-- check if alter table owner fails 
+-- for now, alter db owner is allowed only for admin
+
+create database dbao;
+alter database dbao set owner user user2;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q
new file mode 100644
index 0000000000000000000000000000000000000000..f9049350180eecc3f86dbfc110ec1ca12b8403bc
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q
@@ -0,0 +1,8 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=user1;
+
+-- check if alter table owner fails
+alter database default set owner user user1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q
new file mode 100644
index 0000000000000000000000000000000000000000..de91e91923308f8efef5ac800f9c76d2b11e2fc2
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q
@@ -0,0 +1,6 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set user.name=hive_admin_user;
+set role ADMIN;
+create role all;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q
new file mode 100644
index 0000000000000000000000000000000000000000..42a42f65b28a9de1ce4a95d476f04a8a3c288254
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q
@@ -0,0 +1,6 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set user.name=hive_admin_user;
+set role ADMIN;
+create role default;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q
new file mode 100644
index 0000000000000000000000000000000000000000..0d14cde6d5460cb41c5a6c82b530a22d3582967e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q
@@ -0,0 +1,6 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set user.name=hive_admin_user;
+set role ADMIN;
+create role None;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q
new file mode 100644
index 0000000000000000000000000000000000000000..d5ea284f1474943ec519a81be1ac837fee6b40fd
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q
@@ -0,0 +1,17 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set user.name=hive_admin_user;
+set role ADMIN;
+
+create role testrole;
+show roles;
+drop role TESTROLE;
+show roles;
+create role TESTROLE;
+show roles;
+grant role testROLE to user hive_admin_user;
+set role testrolE;
+set role adMin;
+show roles;
+create role TESTRoLE;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q
new file mode 100644
index 0000000000000000000000000000000000000000..02bbe090cfba71569466c95995e746d5b3c48645
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q
@@ -0,0 +1,7 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=hive_test_user;
+
+-- permanent function creation should fail for non-admin roles
+create function perm_fn as 'org.apache.hadoop.hive.ql.udf.UDFAscii';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q
new file mode 100644
index 0000000000000000000000000000000000000000..8760fa8d8225976dbece6c787f1879fc6fe9de60
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q
@@ -0,0 +1,8 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=hive_test_user;
+
+-- temp function creation should fail for non-admin roles
+create temporary function temp_fn as 'org.apache.hadoop.hive.ql.udf.UDFAscii';
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q
new file mode 100644
index 0000000000000000000000000000000000000000..c904a100c515cef33ed50b34da9cae5762ec2599
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q
@@ -0,0 +1,8 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=hive_test_user;
+
+-- temp macro creation should fail for non-admin roles
+create temporary macro mymacro1(x double) x * x;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q
new file mode 100644
index 0000000000000000000000000000000000000000..a84fe64bd618ea98ca48be3c1067cc9dabfe9df7
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q
@@ -0,0 +1,3 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+-- this test will fail because hive_test_user is not in admin role.
+create role r1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q
new file mode 100644
index 0000000000000000000000000000000000000000..9b1f2ea6c6acb7c8f37dc72c4af46ecee17a8752
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q
@@ -0,0 +1,10 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+-- check create view without select privileges
+create table t1(i int);
+set user.name=user1;
+create view v1 as select * from t1;
+
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q
new file mode 100644
index 0000000000000000000000000000000000000000..1cf74a365d79e6c999aaa167613699ac51bd5261
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q
@@ -0,0 +1,10 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+-- check query without select privilege fails
+create table t1(i int);
+
+set user.name=user1;
+create table t2 as select * from t1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q
new file mode 100644
index 0000000000000000000000000000000000000000..47663c9bb93e3997ba546fe4dc3856d70082b71c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q
@@ -0,0 +1,14 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=user1;
+
+-- check if alter table fails as different user
+create table t1(i int);
+desc t1;
+
+grant all on table t1 to user user2;
+revoke select on table t1 from user user2;
+
+set user.name=user2;
+desc t1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q
new file mode 100644
index 0000000000000000000000000000000000000000..7d47a7b64967bf96a21cca63b1a93b67f76c768c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q
@@ -0,0 +1,7 @@
+set hive.security.authorization.enabled=true;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
+
+-- running a sql query to initialize the authorization - not needed in real HS2 mode
+show tables;
+dfs -ls ${system:test.tmp.dir}/
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q
new file mode 100644
index 0000000000000000000000000000000000000000..64b300c8d9b2fc56cbe007af4531070861602fe9
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q
@@ -0,0 +1,3 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set role ALL;
+SELECT TRANSFORM (*) USING 'cat' AS (key, value) FROM src;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q
new file mode 100644
index 0000000000000000000000000000000000000000..edeae9b71d7acf4f5fd594ed67908e5632a72b29
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q
@@ -0,0 +1,22 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=user1;
+
+-- ensure that drop database cascade works
+create database dba1;
+create table dba1.tab1(i int);
+drop database dba1 cascade;
+
+-- check if drop database fails if the db has a table for which user does not have permission
+create database dba2;
+create table dba2.tab2(i int);
+
+set user.name=hive_admin_user;
+set role ADMIN;
+alter database dba2 set owner user user2;
+
+set user.name=user2;
+show current roles;
+drop database dba2 cascade ;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q
new file mode 100644
index 0000000000000000000000000000000000000000..46d4d0f92c8e3db027861906a26f116d8190b12b
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q
@@ -0,0 +1,27 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=user1;
+
+-- check if changing owner and dropping as other user works
+create database dba1;
+
+set user.name=hive_admin_user;
+set role ADMIN;
+alter database dba1 set owner user user2;
+
+set user.name=user2;
+show current roles;
+drop database dba1;
+
+
+set user.name=user1;
+-- check if dropping db as another user fails
+show current roles;
+create database dba2;
+
+set user.name=user2;
+show current roles;
+
+drop database dba2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q
new file mode 100644
index 0000000000000000000000000000000000000000..a7aa17f5abfcf24ddd2c3104ac50f240af945ae2
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q
@@ -0,0 +1,10 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set user.name=hive_admin_user;
+set role ADMIN;
+show current roles;
+create role r1;
+set role ALL;
+show current roles;
+drop role r1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q
new file mode 100644
index 0000000000000000000000000000000000000000..f05e9458fa804b3779723237f83eef0ef668030e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q
@@ -0,0 +1,11 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/authz_drop_part_1;
+
+-- check drop partition without delete privilege
+create table tpart(i int, j int) partitioned by (k string);
+alter table tpart add partition (k = 'abc') location 'file:${system:test.tmp.dir}/authz_drop_part_1' ;
+set user.name=user1;
+alter table tpart drop partition (k = 'abc');
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q
new file mode 100644
index 0000000000000000000000000000000000000000..d969e39027e99b27ac65cdee2a049a4b123da8e9
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q
@@ -0,0 +1,5 @@
+set hive.security.authorization.enabled=true;
+
+create database db_to_fail;
+
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q
new file mode 100644
index 0000000000000000000000000000000000000000..87719b0043e2e77a138b6d89e4474c7080bd168e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q
@@ -0,0 +1,5 @@
+set hive.security.authorization.enabled=false;
+create database db_fail_to_drop;
+set hive.security.authorization.enabled=true;
+
+drop database db_fail_to_drop;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q
new file mode 100644
index 0000000000000000000000000000000000000000..f3c86b97ce76f0ea6ee186d61319fca6949ce278
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q
@@ -0,0 +1,14 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_allf(i int);
+
+-- grant insert to user2 WITH grant option
+GRANT INSERT ON table_priv_allf TO USER user2 with grant option;
+
+set user.name=user2;
+-- try grant all to user3, without having all privileges
+GRANT ALL ON table_priv_allf TO USER user3;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q
new file mode 100644
index 0000000000000000000000000000000000000000..7808cb3ec7b39c845616a97cdce5ddf6faa4660a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q
@@ -0,0 +1,16 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE  tauth_gdup(i int);
+
+-- It should be possible to revert owners privileges
+revoke SELECT ON tauth_gdup from user user1;
+
+show grant user user1 on table tauth_gdup;
+
+-- Owner already has all privileges granted, another grant would become duplicate
+-- and result in error
+GRANT INSERT ON tauth_gdup TO USER user1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q
new file mode 100644
index 0000000000000000000000000000000000000000..8dc8e45a790750be5b087af4ddf837bd3a9bb627
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q
@@ -0,0 +1,11 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE  table_priv_gfail1(i int);
+
+set user.name=user2;
+-- try grant insert to user3 as user2
+GRANT INSERT ON table_priv_gfail1 TO USER user3;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q
new file mode 100644
index 0000000000000000000000000000000000000000..d51c1c3507eef2c92b32ccb9e7cd62de1f8d735a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q
@@ -0,0 +1,14 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_gfail1(i int);
+
+-- grant insert to user2 WITHOUT grant option
+GRANT INSERT ON table_priv_gfail1 TO USER user2;
+
+set user.name=user2;
+-- try grant insert to user3
+GRANT INSERT ON table_priv_gfail1 TO USER user3;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q
new file mode 100644
index 0000000000000000000000000000000000000000..2fa3cb260b07ecaeb53e4ea93c675e23d992e175
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q
@@ -0,0 +1,11 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+-- check insert without select priv
+create table t1(i int);
+
+set user.name=user1;
+create table user2tab(i int);
+insert into table t1 select * from user2tab;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q
new file mode 100644
index 0000000000000000000000000000000000000000..b9bee4ea40d405cc737f83eb4c251538d5a98f9a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q
@@ -0,0 +1,11 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+-- check insert without select priv
+create table t1(i int);
+
+set user.name=user1;
+create table t2(i int);
+insert into table t2 select * from t1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q
new file mode 100644
index 0000000000000000000000000000000000000000..2a1da23daeb18a5a9ab093e5440c19e33c0ebedd
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q
@@ -0,0 +1,6 @@
+create table if not exists authorization_invalid_v1 (key int, value string);
+grant delete on table authorization_invalid_v1 to user hive_test_user;
+drop table authorization_invalid_v1;
+
+
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q
new file mode 100644
index 0000000000000000000000000000000000000000..9c724085d9901771fde46263fe3880464636f652
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q
@@ -0,0 +1,5 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+
+create table if not exists authorization_invalid_v2 (key int, value string);
+grant index on table authorization_invalid_v2 to user hive_test_user;
+drop table authorization_invalid_v2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q
new file mode 100644
index 0000000000000000000000000000000000000000..8a3300cb2e3788dd9606da368eb87a89b8061c3b
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q
@@ -0,0 +1,10 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=user1;
+
+-- check if alter table fails as different user
+create table t1(i int);
+
+set user.name=user2;
+alter table t1 rename to tnew1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q
new file mode 100644
index 0000000000000000000000000000000000000000..0172c4c74c82d56ffa4e67108a9ef5d2ec4bb0ea
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q
@@ -0,0 +1,10 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=user1;
+
+-- check if alter table fails as different user
+create table t1(i int);
+
+set user.name=user2;
+ALTER TABLE t1 SET SERDEPROPERTIES ('field.delim' = ',');
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q
new file mode 100644
index 0000000000000000000000000000000000000000..2d0e52da008d89ff7e7ebeaa5a00a03e90976d18
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q
@@ -0,0 +1,11 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=user1;
+
+-- check if create table fails as different user
+create table t1(i int);
+
+set user.name=user2;
+drop table t1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q
new file mode 100644
index 0000000000000000000000000000000000000000..76bbab42b3750e0799ca8c6cff2183d4e5c59e3c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q
@@ -0,0 +1,11 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=user1;
+
+-- check if create table fails as different user
+create table t1(i int);
+create view vt1 as select * from t1;
+
+set user.name=user2;
+drop view vt1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q
new file mode 100644
index 0000000000000000000000000000000000000000..bbf3b66970b6a79e42d59ec1d5687d9b714e6576
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q
@@ -0,0 +1,29 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set user.name=hive_admin_user;
+set role ADMIN;
+
+-- the test verifies that authorization is happening with privileges of the current roles
+
+-- grant privileges with grant option for table to role2 
+create role role2;
+grant role role2 to user user2;
+create table tpriv_current_role(i int);
+grant all on table tpriv_current_role to role role2 with grant option;
+
+set user.name=user2;
+-- switch to user2
+
+-- by default all roles should be in current roles, and grant to new user should work
+show current roles;
+grant all on table tpriv_current_role to user user3;
+
+set role role2;
+-- switch to role2, grant should work
+grant all on table tpriv_current_role to user user4;
+show grant user user4 on table tpriv_current_role;
+
+set role PUBLIC;
+-- set role to public, should fail as role2 is not one of the current roles
+grant all on table tpriv_current_role to user user5;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q
new file mode 100644
index 0000000000000000000000000000000000000000..002389f203e25fe190ab740d255eae2046081eba
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q
@@ -0,0 +1 @@
+create role PUBLIC;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q
new file mode 100644
index 0000000000000000000000000000000000000000..69c5a8de8b05f23ad7d37c9ecf19533f54d078b3
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q
@@ -0,0 +1 @@
+drop role PUBLIC;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q
new file mode 100644
index 0000000000000000000000000000000000000000..e19bf370fa0775fe6fee5e09ddc28fd8bfe5f7ca
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q
@@ -0,0 +1,14 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_rfail1(i int);
+
+-- grant insert to user2
+GRANT INSERT ON table_priv_rfail1 TO USER user2;
+
+set user.name=user3;
+-- try dropping the privilege as user3
+REVOKE INSERT ON TABLE table_priv_rfail1 FROM USER user2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q
new file mode 100644
index 0000000000000000000000000000000000000000..4b0cf3286ae71e25c8644511b72da6a9cdf14e73
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q
@@ -0,0 +1,18 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_rfai2(i int);
+
+-- grant insert to user2
+GRANT INSERT ON table_priv_rfai2 TO USER user2;
+GRANT SELECT ON table_priv_rfai2 TO USER user3 WITH GRANT OPTION;
+
+set user.name=user3;
+-- grant select as user3 to user 2
+GRANT SELECT ON table_priv_rfai2 TO USER user2;
+
+-- try dropping the privilege as user3
+REVOKE INSERT ON TABLE table_priv_rfai2 FROM USER user2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q
new file mode 100644
index 0000000000000000000000000000000000000000..a819d204f56b64535b730f274c431a792e2c1ac1
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q
@@ -0,0 +1,12 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set user.name=hive_admin_user;
+set role ADMIN;
+-- this is applicable to any security mode as check is in metastore
+create role role1;
+create role role2;
+grant role role1 to role role2;
+
+-- this will create a cycle
+grant role role2 to role role1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q
new file mode 100644
index 0000000000000000000000000000000000000000..423f030630b6cd91ef1a3ae9bab85f5b023831fd
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q
@@ -0,0 +1,24 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=hive_admin_user;
+set role ADMIN;
+-- this is applicable to any security mode as check is in metastore
+
+create role role1;
+
+create role role2;
+grant role role2 to role role1;
+
+create role role3;
+grant role role3 to role role2;
+
+create role role4;
+grant role role4 to role role3;
+
+create role role5;
+grant role role5 to role role4;
+
+-- this will create a cycle in middle of the hierarchy
+grant role role2 to role role4;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q
new file mode 100644
index 0000000000000000000000000000000000000000..c5c500a71251f776e5743371265033373573673d
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q
@@ -0,0 +1,22 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set user.name=hive_admin_user;
+
+set role ADMIN;
+
+----------------------------------------
+-- role granting with admin option
+-- since user2 doesn't have admin option for role_noadmin, last grant should fail
+----------------------------------------
+
+create role role_noadmin;
+create role src_role_wadmin;
+grant  src_role_wadmin to user user2 with admin option;
+grant  role_noadmin to user user2;
+show role grant user user2;
+
+
+set user.name=user2;
+set role role_noadmin;
+grant  src_role_wadmin to user user3;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q
new file mode 100644
index 0000000000000000000000000000000000000000..d9f4c7cdb850bf40d20ddbb6c7a87798716b1475
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q
@@ -0,0 +1,74 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+set user.name=hive_admin_user;
+show current roles;
+set role ADMIN;
+
+----------
+-- create the following user, role mapping
+-- user1 -> role1 -> role2 -> role3
+----------
+
+create role role1;
+grant role1 to user user1;
+
+create role role2;
+grant role2 to role role1;
+
+create role role3;
+grant role3 to role role2;
+
+
+create table t1(i int);
+grant select on t1 to role role3;
+
+set user.name=user1;
+show current roles;
+select * from t1;
+
+set user.name=hive_admin_user;
+show current roles;
+grant select on t1 to role role2;
+
+
+set user.name=user1;
+show current roles;
+select * from t1;
+
+set user.name=hive_admin_user;
+set role ADMIN;
+show current roles;
+revoke select on table t1 from role role2;
+
+
+create role role4;
+grant role4 to user user1;
+grant role3 to role role4;;
+
+set user.name=user1;
+show current roles;
+select * from t1;
+
+set user.name=hive_admin_user;
+show current roles;
+set role ADMIN;
+
+-- Revoke role3 from hierarchy one at a time and check permissions
+-- after revoking from both, select should fail
+revoke role3 from role role2;
+
+set user.name=user1;
+show current roles;
+select * from t1;
+
+set user.name=hive_admin_user;
+show current roles;
+set role ADMIN;
+revoke role3 from role role4;
+
+set user.name=user1;
+show current roles;
+select * from t1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q
new file mode 100644
index 0000000000000000000000000000000000000000..39871793af398d4f1d0fc907594d42252da9cc1b
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q
@@ -0,0 +1,9 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+-- check query without select privilege fails
+create table t1(i int);
+
+set user.name=user1;
+select * from t1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q
new file mode 100644
index 0000000000000000000000000000000000000000..a4071cd0d4d87f15dff6130b192ce8c351025aee
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q
@@ -0,0 +1,11 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+-- check create view without select privileges
+create table t1(i int);
+create view v1 as select * from t1;
+set user.name=user1;
+select * from v1;
+
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q
new file mode 100644
index 0000000000000000000000000000000000000000..9ba3a82a5608eaba951de8ad569bf31bd76f5dd6
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q
@@ -0,0 +1,6 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+
+-- an error should be thrown if 'set role ' is done for role that does not exist
+
+set role nosuchroleexists;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q
new file mode 100644
index 0000000000000000000000000000000000000000..03f748fcc9b7aad0c0a1b56753d8bd02c565c13c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q
@@ -0,0 +1,16 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set user.name=hive_admin_user;
+set role ADMIN;
+
+-- an error should be thrown if 'set role ' is done for role that does not exist
+
+create role rset_role_neg;
+grant role rset_role_neg to user user2;
+
+set user.name=user2;
+set role rset_role_neg;
+set role public;
+set role nosuchroleexists;;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q
new file mode 100644
index 0000000000000000000000000000000000000000..d8190de950de7f41919eb2230e8062b09edbe049
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q
@@ -0,0 +1,10 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=user1;
+
+-- check if alter table fails as different user
+create table t_show_parts(i int) partitioned by (j string);
+
+set user.name=user2;
+show partitions t_show_parts;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q
new file mode 100644
index 0000000000000000000000000000000000000000..2afe87fc30c9edb38441edf07fc96542f1ff01da
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q
@@ -0,0 +1,3 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+-- This test will fail because hive_test_user is not in admin role
+show principals role1; 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q
new file mode 100644
index 0000000000000000000000000000000000000000..69cea2f2673f0a2f8f463faa06e574ea3ace49fb
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q
@@ -0,0 +1,2 @@
+-- This test will fail because the command is not currently supported in auth mode v1
+show principals role1; 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q
new file mode 100644
index 0000000000000000000000000000000000000000..0fc9fca940c39ae3cbb0946c26e0f2413241aa14
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q
@@ -0,0 +1,3 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+-- This test will fail because hive_test_user is not in admin role
+show roles; 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q
new file mode 100644
index 0000000000000000000000000000000000000000..285600b23a149f7e60ebc19537bc9fe6796f4559
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q
@@ -0,0 +1,9 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+-- check add partition without insert privilege
+create table t1(i int, j int);
+set user.name=user1;
+truncate table t1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q
new file mode 100644
index 0000000000000000000000000000000000000000..d82ac710cc3ba4e7b5f41401529d0b23bb0c535c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q
@@ -0,0 +1,10 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_add_part;
+dfs -touchz ${system:test.tmp.dir}/a_uri_add_part/1.txt;
+dfs -chmod 555 ${system:test.tmp.dir}/a_uri_add_part/1.txt;
+
+create table tpart(i int, j int) partitioned by (k string);
+alter table tpart add partition (k = 'abc') location '${system:test.tmp.dir}/a_uri_add_part/';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q
new file mode 100644
index 0000000000000000000000000000000000000000..d38ba74d9006ab848056ef20a6f0040792e466b5
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q
@@ -0,0 +1,16 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_alterpart_loc_perm;
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_alterpart_loc;
+dfs -touchz ${system:test.tmp.dir}/az_uri_alterpart_loc/1.txt;
+dfs -chmod 555 ${system:test.tmp.dir}/az_uri_alterpart_loc/1.txt;
+
+create table tpart(i int, j int) partitioned by (k string);
+alter table tpart add partition (k = 'abc') location '${system:test.tmp.dir}/az_uri_alterpart_loc_perm/';
+
+alter table tpart partition (k = 'abc') set location '${system:test.tmp.dir}/az_uri_alterpart_loc/';
+
+
+-- Attempt to set partition to location without permissions should fail
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q
new file mode 100644
index 0000000000000000000000000000000000000000..c446b8636fb32c0f693fc8babb29033fb0c6f665
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q
@@ -0,0 +1,13 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_altertab_setloc;
+dfs -touchz ${system:test.tmp.dir}/az_uri_altertab_setloc/1.txt;
+dfs -chmod 555 ${system:test.tmp.dir}/az_uri_altertab_setloc/1.txt;
+
+create table t1(i int);
+
+alter table t1 set location '${system:test.tmp.dir}/az_uri_altertab_setloc/1.txt'
+
+-- Attempt to set location of table to a location without permissions should fail
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q
new file mode 100644
index 0000000000000000000000000000000000000000..c8e1fb43ee317d57b3e378e2de5650addfb5c644
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q
@@ -0,0 +1,11 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_crtab1;
+dfs -touchz ${system:test.tmp.dir}/a_uri_crtab1/1.txt;
+dfs -chmod 555 ${system:test.tmp.dir}/a_uri_crtab1/1.txt;
+
+create table t1(i int) location '${system:test.tmp.dir}/a_uri_crtab_ext';
+
+-- Attempt to create table with dir that does not have write permission should fail
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q
new file mode 100644
index 0000000000000000000000000000000000000000..c8549b4563b2f263c23be8d2454579f5e578e9cc
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q
@@ -0,0 +1,11 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_crtab_ext;
+dfs -touchz ${system:test.tmp.dir}/a_uri_crtab_ext/1.txt;
+dfs -chmod 555 ${system:test.tmp.dir}/a_uri_crtab_ext/1.txt;
+
+create external table t1(i int) location '${system:test.tmp.dir}/a_uri_crtab_ext';
+
+-- Attempt to create table with dir that does not have write permission should fail
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q
new file mode 100644
index 0000000000000000000000000000000000000000..edfdf5a8fc4076774b302fc057556df3619293a1
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q
@@ -0,0 +1,12 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_createdb;
+dfs -touchz ${system:test.tmp.dir}/az_uri_createdb/1.txt;
+dfs -chmod 300 ${system:test.tmp.dir}/az_uri_createdb/1.txt;
+
+create database az_test_db location '${system:test.tmp.dir}/az_uri_createdb/';
+
+-- Attempt to create db for dir without sufficient permissions should fail
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q
new file mode 100644
index 0000000000000000000000000000000000000000..81763916a0b819566f6d46086b173cb36691d341
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q
@@ -0,0 +1,22 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+set hive.test.mode=true;
+set hive.test.mode.prefix=;
+set hive.test.mode.nosamplelist=export_auth_uri;
+
+
+create table export_auth_uri ( dep_id int comment "department id")
+	stored as textfile;
+
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/export_auth_uri/temp;
+dfs -rmr target/tmp/ql/test/data/exports/export_auth_uri;
+
+
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/export_auth_uri/;
+dfs -chmod 555 target/tmp/ql/test/data/exports/export_auth_uri;
+
+export table export_auth_uri to 'ql/test/data/exports/export_auth_uri';
+
+-- Attempt to export to location without sufficient permissions should fail
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q
new file mode 100644
index 0000000000000000000000000000000000000000..4ea4dc0a4747a107bd3193cbcf494925d9a609c7
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q
@@ -0,0 +1,25 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+set hive.test.mode=true;
+set hive.test.mode.prefix=;
+set hive.test.mode.nosamplelist=import_auth_uri;
+
+
+create table import_auth_uri ( dep_id int comment "department id")
+	stored as textfile;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/import_auth_uri/temp;
+dfs -rmr target/tmp/ql/test/data/exports/import_auth_uri;
+export table import_auth_uri to 'ql/test/data/exports/import_auth_uri';
+drop table import_auth_uri;
+
+dfs -touchz target/tmp/ql/test/data/exports/import_auth_uri/1.txt;
+dfs -chmod 555 target/tmp/ql/test/data/exports/import_auth_uri/1.txt;
+
+create database importer;
+use importer;
+
+import from 'ql/test/data/exports/import_auth_uri';
+
+-- Attempt to import from location without sufficient permissions should fail
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q
new file mode 100644
index 0000000000000000000000000000000000000000..1a8f9cb2ad197c07567461652d9403c52eefa862
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q
@@ -0,0 +1,13 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_index;
+dfs -touchz ${system:test.tmp.dir}/az_uri_index/1.txt;
+dfs -chmod 555 ${system:test.tmp.dir}/az_uri_index/1.txt;
+
+
+create table t1(i int);
+create index idt1 on table t1 (i) as 'COMPACT' WITH DEFERRED REBUILD LOCATION '${system:test.tmp.dir}/az_uri_index/';
+
+-- Attempt to use location for index that does not have permissions should fail
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q
new file mode 100644
index 0000000000000000000000000000000000000000..81b6e522c1abb2ae96ac88cebf8ea8aa700b3bfc
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q
@@ -0,0 +1,14 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_insert;
+dfs -touchz ${system:test.tmp.dir}/az_uri_insert/1.txt;
+dfs -chmod 555 ${system:test.tmp.dir}/az_uri_insert/1.txt;
+
+create table t1(i int, j int);
+
+insert overwrite directory '${system:test.tmp.dir}/az_uri_insert/' select * from t1;
+
+-- Attempt to insert into uri without permissions should fail
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q
new file mode 100644
index 0000000000000000000000000000000000000000..0a2fd8919f4555d7ab12dd370673e980025647cb
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q
@@ -0,0 +1,14 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_insert_local;
+dfs -touchz ${system:test.tmp.dir}/az_uri_insert_local/1.txt;
+dfs -chmod 555 ${system:test.tmp.dir}/az_uri_insert_local/1.txt;
+
+create table t1(i int, j int);
+
+insert overwrite local directory '${system:test.tmp.dir}/az_uri_insert_local/' select * from t1;
+
+-- Attempt to insert into uri without permissions should fail
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q
new file mode 100644
index 0000000000000000000000000000000000000000..6af41f0cdaa2338a945414c6ffefd56dab893b7d
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q
@@ -0,0 +1,11 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/authz_uri_load_data;
+dfs -touchz ${system:test.tmp.dir}/authz_uri_load_data/1.txt;
+dfs -chmod 555 ${system:test.tmp.dir}/authz_uri_load_data/1.txt;
+
+create table t1(i int);
+load data inpath 'pfile:${system:test.tmp.dir}/authz_uri_load_data/' overwrite into table t1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q
new file mode 100644
index 0000000000000000000000000000000000000000..d8beac370d4b6dfb96bfc2298009571ab236c326
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q
@@ -0,0 +1,10 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set hive.security.authorization.enabled=true;
+set user.name=user33;
+create database db23221;
+use db23221;
+
+set user.name=user44;
+create table twew221(a string);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q
new file mode 100644
index 0000000000000000000000000000000000000000..bfd31652377748004a9318aecdaff72e42bcd184
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q
@@ -0,0 +1 @@
+grant role PUBLIC to user hive_test_user;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q
new file mode 100644
index 0000000000000000000000000000000000000000..2b29822371b192c075460c7f21390d99ac6ac5df
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q
@@ -0,0 +1 @@
+revoke role PUBLIC from user hive_test_user;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q
index 048a8fd5cfb8e3fdc41178eeb4ec4df054808d8b..6bebb8942d613ea326397348e37f8b8e2456a65b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q
@@ -1,19 +1,19 @@
 CREATE TABLE srcbucket_mapjoin_part (key int, value string) 
   partitioned by (ds string) CLUSTERED BY (key) INTO 3 BUCKETS
   STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket20.txt' 
+load data local inpath '../../data/files/srcbucket20.txt' 
   INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket21.txt' 
+load data local inpath '../../data/files/srcbucket21.txt' 
   INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket22.txt' 
+load data local inpath '../../data/files/srcbucket22.txt' 
   INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
 
 CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string)
   partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS
   STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket22.txt'
+load data local inpath '../../data/files/srcbucket22.txt'
   INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket23.txt'
+load data local inpath '../../data/files/srcbucket23.txt'
   INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
 
 -- The number of buckets in the 2 tables above (being joined later) dont match.
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q
index 9478a2f1b989c369a93d4bd39ee1b947cfdd6fc8..802fcd903c0acb44803417c510bb0a335dca8066 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q
@@ -8,10 +8,10 @@ into 2 BUCKETS stored as textfile;
 create table table2(key string, value string) clustered by (value, key)
 into 2 BUCKETS stored as textfile;
 
-load data local inpath '../data/files/T1.txt' overwrite into table table1;
+load data local inpath '../../data/files/T1.txt' overwrite into table table1;
 
-load data local inpath '../data/files/T1.txt' overwrite into table table2;
-load data local inpath '../data/files/T2.txt' overwrite into table table2;
+load data local inpath '../../data/files/T1.txt' overwrite into table table2;
+load data local inpath '../../data/files/T2.txt' overwrite into table table2;
 
 set hive.optimize.bucketmapjoin = true;
 set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q
index 69afe0ae700ede96ac8b218bc06b0f0f1daacc2e..ac5abebb0b4b6b55be2897a9afd4280ef84063f9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q
@@ -8,13 +8,13 @@ into 2 BUCKETS stored as textfile;
 create table table2(key string, value string) clustered by (value, key)
 into 2 BUCKETS stored as textfile;
 
-load data local inpath '../data/files/T1.txt' overwrite into table table1 partition (ds='1');
-load data local inpath '../data/files/T2.txt' overwrite into table table1 partition (ds='1');
+load data local inpath '../../data/files/T1.txt' overwrite into table table1 partition (ds='1');
+load data local inpath '../../data/files/T2.txt' overwrite into table table1 partition (ds='1');
 
-load data local inpath '../data/files/T1.txt' overwrite into table table1 partition (ds='2');
+load data local inpath '../../data/files/T1.txt' overwrite into table table1 partition (ds='2');
 
-load data local inpath '../data/files/T1.txt' overwrite into table table2;
-load data local inpath '../data/files/T2.txt' overwrite into table table2;
+load data local inpath '../../data/files/T1.txt' overwrite into table table2;
+load data local inpath '../../data/files/T2.txt' overwrite into table table2;
 
 set hive.optimize.bucketmapjoin = true;
 set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern1.q
deleted file mode 100644
index 0ff44779657ee86579bef66615381e93f81b3589..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern1.q
+++ /dev/null
@@ -1,2 +0,0 @@
-EXPLAIN
-SELECT x.key, x.value as key FROM SRC x CLUSTER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q
index af923504c8c4f1cf61c0fda2ba68e795f8ca2fbf..b4887c41158545eb1fa2c837ce6721b13889c640 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q
@@ -3,12 +3,12 @@ DROP TABLE Employee_Part;
 CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string)
 row format delimited fields terminated by '|'  stored as textfile;
 
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK');
 
 -- dynamic partitioning syntax
 explain 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q
index d9725ddc7fdf6a092a8c690ff7dc13a739d28fbd..2f8e9271ddd3450a1e0365e666fb1dfc72701481 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q
@@ -3,12 +3,12 @@ DROP TABLE Employee_Part;
 CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string)
 row format delimited fields terminated by '|'  stored as textfile;
 
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK');
 
 -- don't specify all partitioning keys
 explain 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q
index eb73962ba133cf2add4a42551e08bea19192aa5d..34f91fc8d1de846af0c0b6d1961139f08a34cf20 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q
@@ -3,12 +3,12 @@ DROP TABLE Employee_Part;
 CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string)
 row format delimited fields terminated by '|'  stored as textfile;
 
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK');
 
 -- specify invalid values for the partitioning keys
 explain
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q
index dbfaaecbdf614f4d3c6b96c09a77638d7c04455d..49d89dd12132b83a0672e999f7be8aa37ace34ed 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q
@@ -3,12 +3,12 @@ DROP TABLE Employee_Part;
 CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string)
 row format delimited fields terminated by '|'  stored as textfile;
 
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK');
-LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK');
+LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK');
 
 -- specify partitioning clause multiple times
 explain
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q
index ca8548958fcd661b0de001398414e010a22282ae..a4e0056bff370afefd6707979012d600f5661661 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q
@@ -13,7 +13,7 @@ CREATE TABLE UserVisits_web_text_none (
   avgTimeOnSite int)
 row format delimited fields terminated by '|'  stored as textfile;
 
-LOAD DATA LOCAL INPATH "../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none;
+LOAD DATA LOCAL INPATH "../../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none;
 
 explain 
 analyze table UserVisits_web_text_none compute statistics for columns destIP;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q
index 5bbd70d86b297ed727fc46ece3f375ac41c47e01..85a5f0a021940244124081af9033da926edd913a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q
@@ -8,7 +8,7 @@ CREATE TABLE table_complex_type (
        d MAP<STRING,ARRAY<STRING>>
        ) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table_complex_type;
+LOAD DATA LOCAL INPATH '../../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table_complex_type;
 
 
 explain 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q
index ca8548958fcd661b0de001398414e010a22282ae..a4e0056bff370afefd6707979012d600f5661661 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q
@@ -13,7 +13,7 @@ CREATE TABLE UserVisits_web_text_none (
   avgTimeOnSite int)
 row format delimited fields terminated by '|'  stored as textfile;
 
-LOAD DATA LOCAL INPATH "../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none;
+LOAD DATA LOCAL INPATH "../../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none;
 
 explain 
 analyze table UserVisits_web_text_none compute statistics for columns destIP;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q
new file mode 100644
index 0000000000000000000000000000000000000000..c314a940f95c2f47387998e21df93b40eb89cb8c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q
@@ -0,0 +1,8 @@
+
+compile `import org.apache.hadoop.hive.ql.exec.UDF \;
+public class Pyth extsfgsfgfsends UDF {
+  public double evaluate(double a, double b){
+    return Math.sqrt((a*a) + (b*b)) \;
+  }
+} ` AS GROOVY NAMED Pyth.groovy;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q
new file mode 100644
index 0000000000000000000000000000000000000000..5974811280350e375156146da481f508074ab541
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q
@@ -0,0 +1,7 @@
+create table tab_int(a int);
+
+-- insert some data
+LOAD DATA LOCAL INPATH "../../data/files/int.txt" INTO TABLE tab_int;
+
+-- compute stats should raise an error since the number of bit vectors > 1024
+select compute_stats(a, 10000) from tab_int;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q
new file mode 100644
index 0000000000000000000000000000000000000000..3b71e00b2eaa2f70b960445857cd6340743be2ae
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q
@@ -0,0 +1 @@
+create function default.badfunc as 'my.nonexistent.class';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q
new file mode 100644
index 0000000000000000000000000000000000000000..ae95391edd3e54faa6961c0456d5322632e827ed
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q
@@ -0,0 +1 @@
+create function nonexistentdb.badfunc as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q
new file mode 100644
index 0000000000000000000000000000000000000000..2083064593299ba100b6b9086cd076783f356426
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q
@@ -0,0 +1 @@
+create function default.badfunc as 'java.lang.String';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q
new file mode 100644
index 0000000000000000000000000000000000000000..2160b4719662ba3e03ae60cabe04f9ab88749661
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q
@@ -0,0 +1,4 @@
+explain
+with q1 as ( select key from q2 where key = '5'),
+q2 as ( select key from q1 where key = '5')
+select * from (select key from q1) a;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q
new file mode 100644
index 0000000000000000000000000000000000000000..e52a1d97db801219298d3383bc53aa755e5d6fb0
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q
@@ -0,0 +1 @@
+select * from (with q1 as ( select key from q2 where key = '5') select * from q1) a;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal1.q
deleted file mode 100644
index b7fac0d3df346c23e309a57c1ef1d4b90a57a1f4..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal1.q
+++ /dev/null
@@ -1,2 +0,0 @@
--- Not in YYYY-MM-DD format
-SELECT DATE '2001-1-1' FROM src LIMIT 2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q
new file mode 100644
index 0000000000000000000000000000000000000000..1c658c79b99ea7966856f5d53641eed2075f316a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q
@@ -0,0 +1,6 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+drop database if exists drop_nodblock;
+create database drop_nodblock;
+lock database drop_nodblock shared;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q
new file mode 100644
index 0000000000000000000000000000000000000000..ef4b323f063b6bcd3107593abbeeb3f852f1cf7f
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q
@@ -0,0 +1,6 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+drop database if exists drop_nodbunlock;
+create database drop_nodbunlock;
+unlock database drop_nodbunlock;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q
new file mode 100644
index 0000000000000000000000000000000000000000..4a0c6c25c67c7305a8c96f1a898774accde3e1ce
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q
@@ -0,0 +1,6 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+drop table if exists drop_notablelock;
+create table drop_notablelock (c int);
+lock table drop_notablelock shared;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q
new file mode 100644
index 0000000000000000000000000000000000000000..0b00046579f43fa73a442450c5d78169e014c5de
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q
@@ -0,0 +1,6 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+drop table if exists drop_notableunlock;
+create table drop_notableunlock (c int);
+unlock table drop_notableunlock;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q
index 7b0c92311a11a33835b8c513b6b8583316fda1c9..0bd6985e031b64d23d8fc6c7f1b56c8db95011ec 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q
@@ -1,4 +1,4 @@
 
-ADD JAR ../data/files/TestSerDe.jar;
-DELETE JAR ../data/files/TestSerDe.jar;
+ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar;
+DELETE JAR ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar;
 CREATE TABLE DELETEJAR(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' STORED AS TEXTFILE;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q
new file mode 100644
index 0000000000000000000000000000000000000000..892ef00e3f86c279ffaaef572cf51b5cd46fe471
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q
@@ -0,0 +1,3 @@
+set hive.exec.drop.ignorenonexistent=false;
+-- Can't use DROP FUNCTION if the function doesn't exist and IF EXISTS isn't specified
+drop function nonexistent_function;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q
deleted file mode 100644
index 4d238d73a91167d4a4b4403825d3b5a137d269c9..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q
+++ /dev/null
@@ -1,11 +0,0 @@
-create table ptestfilter (a string, b int) partitioned by (c string, d int);
-describe ptestfilter;
-
-alter table ptestfilter add partition (c='US', d=1);
-alter table ptestfilter add partition (c='US', d=2);
-show partitions ptestfilter;
-
-alter table ptestfilter drop partition (c='US', d<'2');
-
-
-
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q
index 0be2e71c94dffd99e4fd277640e1233ff5776e32..0ad99d100dc07f3200fa0edf8f41fb9816952da1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q
@@ -8,7 +8,7 @@ create table source_table like srcpart;
 
 create table dest_table like srcpart;
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE source_table partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE source_table partition(ds='2008-04-08', hr=11);
 
 -- Tests creating dynamic partitions with characters not in the whitelist (i.e. 9)
 -- If the directory is not empty the hook will throw an error, instead the error should come from the metastore
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q
index e4f0daca92bd19f2e5546d39a5d586a1a4bf8cca..ca60d047efdd59af9f27f8e8b651f45e2329fe6a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q
@@ -3,8 +3,8 @@ CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRIN
 SHOW PARTITIONS exchange_part_test1;
 SHOW PARTITIONS exchange_part_test2;
 
-ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='h1');
-ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='h2');
+ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='h1');
+ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='h2');
 SHOW PARTITIONS exchange_part_test1;
 SHOW PARTITIONS exchange_part_test2;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q
index 4d1e0a62a431f9c829cce35e12cb92ed813d10dc..7083edc32b98d26a8d345c0ffe96a27e5bf06771 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q
@@ -8,5 +8,5 @@ ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05');
 SHOW PARTITIONS exchange_part_test1;
 SHOW PARTITIONS exchange_part_test2;
 
--- exchange_part_test2 table partition (ds='2013-04-05') already exists thus this query will fail
+-- exchange_part_test1 table partition (ds='2013-04-05') already exists thus this query will fail
 alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q
index 23777db3ea93df9f6e6aa0df4af6675ba39a0ba3..6dfe81a8b05683787ee1c01a61407f1892483e0f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q
@@ -9,5 +9,5 @@ ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='3');
 SHOW PARTITIONS exchange_part_test1;
 SHOW PARTITIONS exchange_part_test2;
 
--- exchange_part_test2 table partition (ds='2013-04-05', hr='3') already exists thus this query will fail
+-- exchange_part_test1 table partition (ds='2013-04-05') already exists thus this query will fail
 alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q
index 350bf248acc9c5fb1ead296bad1f8b4c5e6c93c4..60671e52e05d5b9bca89fe881d6b5a1508f6d114 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q
@@ -9,5 +9,5 @@ ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='1');
 SHOW PARTITIONS exchange_part_test1;
 SHOW PARTITIONS exchange_part_test2;
 
--- exchange_part_test2 table partition (ds='2013-04-05', hr='1') already exists thus this query will fail
+-- exchange_part_test2 table partition (ds='2013-04-05') already exists thus this query will fail
 alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q
index 81944b333085345928ea836dc01a7e9341d99bc8..38c0eda2368bde51025571afa8a6e6209c540eb8 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q
@@ -2,5 +2,5 @@ CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING);
 CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING);
 SHOW PARTITIONS exchange_part_test1;
 
--- exchange_part_test1 partition (ds='2013-04-05') does not exist thus this query will fail
+-- exchange_part_test2 partition (ds='2013-04-05') does not exist thus this query will fail
 alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q
index d86ecd5785d02a37fa214c48dd01b3b1c3d2f4e4..6ffc33acb92ecaa0bdc3d96aa48268357699f3d2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q
@@ -4,9 +4,9 @@ set hive.test.mode.prefix=;
 create table exim_department ( dep_id int comment "department id")
 	stored as textfile
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;	
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;	
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'nosuchschema://nosuchauthority/ql/test/data/exports/exim_department';
 drop table exim_department;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q
index 5f3223152f7666c5864d0caf9cdd08e2f83fedd3..970e6463e24a58fb58546adbcbd30c3fb9defef9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q
@@ -4,9 +4,9 @@ set hive.test.mode.prefix=;
 create table exim_department ( dep_id int comment "department id")
 	stored as textfile
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;	
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;	
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -16,9 +16,9 @@ use importer;
 create table exim_department ( dep_id int comment "department identifier")
 	stored as textfile
 	tblproperties("maker"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;	
+load data local inpath "../../data/files/test.dat" into table exim_department;	
 import from 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q
index d7204dc478d25857baa2484a11f8e9202953dbaf..358918363d830cd3ac3d0ae85d7eca75e53ab305 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q
@@ -6,16 +6,16 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -27,12 +27,12 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "iso code", emp_state string comment "free-form text")
 	stored as textfile	
 	tblproperties("maker"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");			
 import from 'ql/test/data/exports/exim_employee';
 describe extended exim_employee;
 select * from exim_employee;
 drop table exim_employee;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q
index 6cd7eda455ee6d4ad18547ea499a3b669df0823c..45268c21c00e3b95dc795af896ffe95314b7aba1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q
@@ -4,9 +4,9 @@ set hive.test.mode.prefix=;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;	
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;	
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -18,6 +18,6 @@ create table exim_department ( dep_key int comment "department id")
 	tblproperties("creator"="krishna");
 import from 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q
index 7f3f577c433b3b20e38c474627bb9c23c392829e..cad6c90fd316ed2e531eb4a00b99e9868724cf81 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q
@@ -4,9 +4,9 @@ set hive.test.mode.prefix=;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;	
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;	
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -18,6 +18,6 @@ create table exim_department ( dep_id int comment "department id", dep_name stri
 	tblproperties("creator"="krishna");
 import from 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q
index d3ec9fff82bf94037e2dce8955c2a72d3578d427..f5f904f42af5a0e385e9c77d26c8aefa4f004226 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q
@@ -4,9 +4,9 @@ set hive.test.mode.prefix=;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;	
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;	
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -18,6 +18,6 @@ create table exim_department ( dep_id bigint comment "department id")
 	tblproperties("creator"="krishna");
 import from 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q
index 1cc691fc2912dd0bafadb7e1dc05ff797b5f6a5b..c56329c03f89f3a9b5bcb603605810148b42d8c8 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q
@@ -4,9 +4,9 @@ set hive.test.mode.prefix=;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;	
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;	
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -18,6 +18,6 @@ create table exim_department ( dep_id int comment "department id")
 	tblproperties("creator"="krishna");
 import from 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q
index 27830ad5f93ddcd0a393ea7b43b5185e01713a00..afaedcd37bf7216b5e4990226178476de0597303 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q
@@ -4,9 +4,9 @@ set hive.test.mode.prefix=;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;	
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;	
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -21,6 +21,6 @@ create table exim_department ( dep_id int comment "department id")
 	tblproperties("creator"="krishna");
 import from 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q
index d85048a97a87611ef8cbb7bc89986daee4cd975f..230b28c402cc555885c676a5a4812f078807ebba 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q
@@ -4,9 +4,9 @@ set hive.test.mode.prefix=;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;	
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;	
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -19,6 +19,6 @@ create table exim_department ( dep_id int comment "department id")
 	tblproperties("creator"="krishna");
 import from 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q
index 84b3786a161c69f52e8e045c567ce128e4af6735..c2e00a966346801cd5a13d6754a6c4facadfde36 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q
@@ -4,9 +4,9 @@ set hive.test.mode.prefix=;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;	
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;	
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -23,6 +23,6 @@ create table exim_department ( dep_id int comment "department id")
 	tblproperties("creator"="krishna");
 import from 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q
index eaf9c579d51d4762d8dc15a16d1c1ff5e35f9820..a6586ead0c23fc6bb2bada097e2b89dd23728fe5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q
@@ -4,9 +4,9 @@ set hive.test.mode.prefix=;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;	
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;	
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -19,6 +19,6 @@ create table exim_department ( dep_id int comment "department id")
 	tblproperties("creator"="krishna");
 import from 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q
index 092fd779541c3c7bc84643495238084519ea1f2e..990a686ebeea6454d54bb70d18e41c3c99e27e3c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q
@@ -5,9 +5,9 @@ create table exim_department ( dep_id int comment "department id")
 	clustered by (dep_id) sorted by (dep_id desc) into 10 buckets
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;	
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;	
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -20,6 +20,6 @@ create table exim_department ( dep_id int comment "department id")
 	tblproperties("creator"="krishna");
 import from 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q
index 05de3d77b07b8617a2cad219529d11477ec83be1..02537ef022d827b88166bd4c6d7f87617d3c555e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q
@@ -4,9 +4,9 @@ set hive.test.mode.prefix=;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;		
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -18,7 +18,7 @@ create table exim_department ( dep_id int comment "department id")
 	tblproperties("creator"="krishna");
 import from 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
 	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q
index dc194ca814ee4a5082d742cebb8e8bd07bab3bc4..897c6747354b72ed0de55ce57d0d78b7d7f845d9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q
@@ -4,9 +4,9 @@ set hive.test.mode.prefix=;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;		
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -19,7 +19,7 @@ create table exim_department ( dep_id int comment "department id")
 	tblproperties("creator"="krishna");
 import from 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
 	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q
index e233707cc4db2e8f8d22d2f1793da4bc8073982c..12013e5ccfc49b383508fea8c3667be928850a29 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q
@@ -5,9 +5,9 @@ create table exim_department ( dep_id int comment "department id")
 	partitioned by (dep_org string)
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department partition (dep_org="hr");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department partition (dep_org="hr");		
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -19,7 +19,7 @@ create table exim_department ( dep_id int comment "department id")
 	tblproperties("creator"="krishna");
 import from 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
 	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q
index a10788e3e3b6bbf64075cbd7be4e898a5a7e2498..d8d2b8008c9ee1dd6cede8cf957a163c1847005c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q
@@ -5,9 +5,9 @@ create table exim_department ( dep_id int comment "department id")
 	partitioned by (dep_org string)
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department partition (dep_org="hr");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department partition (dep_org="hr");		
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -20,7 +20,7 @@ create table exim_department ( dep_id int comment "department id")
 	tblproperties("creator"="krishna");
 import from 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
 	
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q
index cc4a56ca34b88b9f8c1a845e96f569af1c9c7560..82dcce945595840f5ac6ba372208bd41eafc2346 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q
@@ -6,16 +6,16 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -25,6 +25,6 @@ import table exim_employee partition (emp_country="us") from 'ql/test/data/expor
 describe extended exim_employee;
 select * from exim_employee;
 drop table exim_employee;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q
index 140e3bb3b1bbcb26b109943d481108e978990167..d92efeb9a70ef8549450c7e911bd59f4a9b26da3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q
@@ -6,16 +6,16 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -25,6 +25,6 @@ import table exim_employee partition (emp_country="us", emp_state="kl") from 'ql
 describe extended exim_employee;
 select * from exim_employee;
 drop table exim_employee;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q
index 048befe4d3d435e7183e38eb203d13a8f5d5ad9e..12d827b9c838c8fbd07675f021a0c3960a5bad4a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q
@@ -4,9 +4,9 @@ set hive.test.mode.prefix=;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;		
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -17,7 +17,7 @@ create  table exim_department ( dep_id int comment "department id")
 	stored as textfile
 	tblproperties("creator"="krishna");
 import external table exim_department from 'ql/test/data/exports/exim_department';
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 drop table exim_department;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q
index 89cbb9ecd808661b5c9a21bebfb412deac7337e1..726dee53955af2e757a5f8311599e360e7956850 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q
@@ -4,17 +4,17 @@ set hive.test.mode.prefix=;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;		
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
 create database importer;
 use importer;
 
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_department/temp;
-dfs -rmr ../build/ql/test/data/tablestore/exim_department;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_department;
 
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile
@@ -22,9 +22,9 @@ create table exim_department ( dep_id int comment "department id")
 	tblproperties("creator"="krishna");
 import table exim_department from 'ql/test/data/exports/exim_department'
 	location 'ql/test/data/tablestore2/exim_department';
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/tablestore/exim_department;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_department;
 
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q
index 0cbfc85258d2c539ced62a883a85237bc1e3505e..d187c782020341843989759bb8bc7a40bd34ccdd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q
@@ -6,16 +6,16 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -29,7 +29,7 @@ create table exim_employee ( emp_id int comment "employee id")
 	tblproperties("creator"="krishna");
 import external table exim_employee partition (emp_country="us", emp_state="tn") 
 	from 'ql/test/data/exports/exim_employee';
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 drop table exim_employee;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q
index d9ab0cf0e4e50d68e4d4da935ed14bef303ad76c..b818686f773dfe328c6192f5883a5804566ff72f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q
@@ -5,8 +5,8 @@ create table exim_department ( dep_id int) stored as textfile;
 
 set hive.security.authorization.enabled=true;
 
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 
 set hive.security.authorization.enabled=false;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q
index 2dbd534074fa865d4763f7ee407e9f662f4a1f42..4acefb9f0ae1242c9bfe0513fc43b90da59c8b8b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q
@@ -2,9 +2,9 @@ set hive.test.mode=true;
 set hive.test.mode.prefix=;
 
 create table exim_department ( dep_id int) stored as textfile;
-load data local inpath "../data/files/test.dat" into table exim_department;
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -18,5 +18,5 @@ import from 'ql/test/data/exports/exim_department';
 set hive.security.authorization.enabled=false;
 drop table exim_department;
 drop database importer;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q
index ccbcee3698daeec73c2c36b7c39f0678a1c709fc..467014e4679f674136ceb119ef2c19199956ab12 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q
@@ -7,10 +7,10 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -26,6 +26,6 @@ set hive.security.authorization.enabled=true;
 import from 'ql/test/data/exports/exim_employee';
 set hive.security.authorization.enabled=false;
 
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 drop table exim_employee;
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q
index 50bfe005c427877a5f754fbe2f3ac8d0c5e26167..595fa7e76495272f059eebce794d4e46cda89ed9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q
@@ -3,9 +3,9 @@ set hive.test.mode.prefix=;
 set hive.test.mode.nosamplelist=exim_department,exim_employee;
 
 create table exim_department ( dep_id int) stored as textfile;
-load data local inpath "../data/files/test.dat" into table exim_department;
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -19,5 +19,5 @@ set hive.security.authorization.enabled=false;
 select * from exim_department;
 drop table exim_department;
 drop database importer;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q
index 9f44f225e955a9912d6a4bbb96378d4606fc77a1..82230f782eac35a945da053f10916838faf62bc2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q
@@ -2,6 +2,6 @@ CREATE TABLE fetchtask_ioexception (
   KEY STRING,
   VALUE STRING) STORED AS SEQUENCEFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/kv1_broken.seq' OVERWRITE INTO TABLE fetchtask_ioexception;
+LOAD DATA LOCAL INPATH '../../data/files/kv1_broken.seq' OVERWRITE INTO TABLE fetchtask_ioexception;
 
 SELECT * FROM fetchtask_ioexception;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q
new file mode 100644
index 0000000000000000000000000000000000000000..286cf1afb491b9e15825ed53acacb4c6b17b5466
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q
@@ -0,0 +1,13 @@
+dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_file_with_header_footer_negative/;
+
+dfs -copyFromLocal ../data/files/header_footer_table_1 hdfs:///tmp/test_file_with_header_footer_negative/header_footer_table_1;
+
+dfs -copyFromLocal ../data/files/header_footer_table_2 hdfs:///tmp/test_file_with_header_footer_negative/header_footer_table_2;
+
+CREATE EXTERNAL TABLE header_footer_table_1 (name string, message string, id int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LOCATION 'hdfs:///tmp/test_file_with_header_footer_negative/header_footer_table_1' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="200");
+
+SELECT * FROM header_footer_table_1;
+
+DROP TABLE header_footer_table_1;
+
+dfs -rmr hdfs:///tmp/test_file_with_header_footer_negative;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q
index 1cdaffd1f31a307fd6c2ae3b3047370e30f65339..1ab828c8beae4bb2b4690ef58c9173e0ac20f392 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q
@@ -1,6 +1,6 @@
 -- begin part(string, int) pass(string, string)
 CREATE TABLE tab1 (id1 int,id2 string) PARTITIONED BY(month string,day int) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' ;
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='second');
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='second');
 
 select * from tab1;
 drop table tab1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q
new file mode 100644
index 0000000000000000000000000000000000000000..49e6a092fc12739c8357d1125a4687c10fcb3977
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q
@@ -0,0 +1,4 @@
+create table tab1(c int) partitioned by (i int);
+alter table tab1 add partition(i = "some name");
+
+drop table tab1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q
new file mode 100644
index 0000000000000000000000000000000000000000..50f486e6245cff88b25c4cc331a80dfeafe0c99a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q
@@ -0,0 +1,3 @@
+create table tab1(s string) PARTITIONED BY(dt date, st string);
+alter table tab1 add partition (dt=date 'foo', st='foo');
+drop table tab1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q
index 7d003e3e4b4408ac5c3668d79d244fd6f2b30200..5bb889c027743928568599cbe5ca3123132105b9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 drop index src_index on src;
 
 CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q
index d79674539a1b2b24afcac266cb5fc5d7a107cded..c6600e69b6a7c2d1465ecc5b9df4f0c89d30d9a3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 drop index src_index on src;
 
 CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q
new file mode 100644
index 0000000000000000000000000000000000000000..c20c168a887c25c06b5f7cd0d86c57fec8fd8361
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q
@@ -0,0 +1,9 @@
+DROP TABLE if exists insert_into5_neg;
+
+CREATE TABLE insert_into5_neg (key int, value string) TBLPROPERTIES ("immutable"="true");
+
+INSERT INTO TABLE insert_into5_neg SELECT * FROM src LIMIT 100;
+
+INSERT INTO TABLE insert_into5_neg SELECT * FROM src LIMIT 100;
+
+DROP TABLE insert_into5_neg;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q
new file mode 100644
index 0000000000000000000000000000000000000000..a92ee5ca94a33976b20ab9ad840e0c3d816ed68e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q
@@ -0,0 +1,17 @@
+DROP TABLE IF EXISTS insert_into6_neg;
+
+CREATE TABLE insert_into6_neg (key int, value string)
+  PARTITIONED BY (ds string) TBLPROPERTIES("immutable"="true") ;
+
+INSERT INTO TABLE insert_into6_neg PARTITION (ds='1')
+  SELECT * FROM src LIMIT 100;
+
+INSERT INTO TABLE insert_into6_neg PARTITION (ds='2')
+  SELECT * FROM src LIMIT 100;
+
+SELECT COUNT(*) from insert_into6_neg;
+
+INSERT INTO TABLE insert_into6_neg PARTITION (ds='1')
+  SELECT * FROM src LIMIT 100;
+
+DROP TABLE insert_into6_neg;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q
index cbf65c4ac69fe5a0402f89421b7567c317e7a845..a8f77c28a82519bad1539ee9f738d65ecb765bd5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q
@@ -4,6 +4,6 @@ create table srcpart_dp like srcpart;
 
 create table destpart_dp like srcpart;
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_dp partition(ds='2008-04-08', hr=11);
 
 insert overwrite table destpart_dp partition (ds='2008-04-08', hr) if not exists select key, value, hr from srcpart_dp where ds='2008-04-08';
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q
new file mode 100644
index 0000000000000000000000000000000000000000..ba7d164c77155ab114ac199a427852695553fa73
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q
@@ -0,0 +1,2 @@
+drop table invalid_char_length_1;
+create table invalid_char_length_1 (c1 char(1000000));
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q
new file mode 100644
index 0000000000000000000000000000000000000000..866b43d31273dcd6e6b9568543c3cf909fa4ff64
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q
@@ -0,0 +1 @@
+select cast(value as char(100000)) from src limit 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q
new file mode 100644
index 0000000000000000000000000000000000000000..481b630d20489f968077fac87bc20bcecbc4906b
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q
@@ -0,0 +1,3 @@
+drop table invalid_char_length_3;
+create table invalid_char_length_3 (c1 char(0));
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_columns.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_columns.q
deleted file mode 100644
index 14b3409cb4cc43bf3a9a406bfbe931eafe5389fa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_columns.q
+++ /dev/null
@@ -1,4 +0,0 @@
-ADD JAR ../data/files/TestSerDe.jar;
-CREATE TABLE DELETEJAR(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' 
-STORED AS TEXTFILE
-TBLPROPERTIES('columns'='valid_colname,invalid.colname');
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q
new file mode 100644
index 0000000000000000000000000000000000000000..e39a38e2fcd47ebab1bf682b2a249e28e98bb38a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q
@@ -0,0 +1,3 @@
+explain select *
+from src s1 , 
+src s2 on s1.key = s2.key;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q
new file mode 100644
index 0000000000000000000000000000000000000000..c0da913c288125f5c9bd182591ee9ad5b3bad093
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q
@@ -0,0 +1,6 @@
+
+
+explain select s1.key, s2.key
+from src s1, src s2
+where key = s2.key
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q
new file mode 100644
index 0000000000000000000000000000000000000000..8e219637eb0c6f829264de9e809b90b1da71e57b
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q
@@ -0,0 +1,5 @@
+
+explain select s1.key, s2.key
+from src s1, src s2
+where INPUT__FILE__NAME = s2.INPUT__FILE__NAME
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q
new file mode 100644
index 0000000000000000000000000000000000000000..d59394544ccf0ef614c97b9016286b86f5e0cc34
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q
@@ -0,0 +1,7 @@
+set hive.limit.query.max.table.partition=1;
+
+explain select * from srcpart limit 1;
+select * from srcpart limit 1;
+
+explain select * from srcpart;
+select * from srcpart;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q
new file mode 100644
index 0000000000000000000000000000000000000000..0afd4a965ab9467b8593f9540fa8bec2f56068a4
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q
@@ -0,0 +1,18 @@
+set hive.exec.dynamic.partition=true;
+set hive.exec.dynamic.partition.mode=nonstrict;
+set hive.stats.autogather=true;
+set hive.compute.query.using.stats=true;
+
+create table part (c int) partitioned by (d string);
+insert into table part partition (d)
+select hr,ds from srcpart;
+
+set hive.limit.query.max.table.partition=1;
+
+explain select count(*) from part;
+select count(*) from part;
+
+set hive.compute.query.using.stats=false;
+
+explain select count(*) from part;
+select count(*) from part;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q
index f86cd92d9d4c724b37d193e24bfffaddb5a1e7f2..eb72d940a53926d68bbfe96dfe2d53170221cdb6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q
@@ -1,4 +1,4 @@
 create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile;
 alter table hive_test_src add partition (pcol1 = 'test_part');
 set hive.security.authorization.enabled=true;
-load data local inpath '../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part');
+load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part');
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q
index 387aaed9a1e50440d2ce50c416e9154f16cb85ed..75a5216e00d821a66c3872d92547a35404d68bbc 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q
@@ -2,4 +2,4 @@
 CREATE TABLE non_native2(key int, value string) 
 STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler';
 
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE non_native2;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE non_native2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q
index d807c698777f1e66f73a279d001b9aafd08b3f83..32653631ad6a49f08f99c60643eb66c7f73321f2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q
@@ -1,3 +1,3 @@
 create table hive_test_src ( col1 string ) stored as textfile;
 set hive.security.authorization.enabled=true;
-load data local inpath '../data/files/test.dat' overwrite into table hive_test_src ;
+load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q
index c409d5a94a9ce8c9ad65a1d173a2c189f2f99163..315988dc0a959417ac70a1102011e648e9202123 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q
@@ -1,3 +1,3 @@
 create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile;
 set hive.security.authorization.enabled=true;
-load data local inpath '../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part');
+load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part');
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q
index 356c16a664386c2e282481f9ea6b747e6f111bd7..81517991b26fa20120838b474fb3c49c1c7b022c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q
@@ -1,2 +1,2 @@
 create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile;
-load data local inpath '../data/files/test.dat' into table hive_test_src;
+load data local inpath '../../data/files/test.dat' into table hive_test_src;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q
index eed5651cbf20b0001d425ddc51dc3f77fac70404..c56f0d408d4addfcbce48b6dc278b4ac4e42b9ba 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q
@@ -4,4 +4,4 @@ set hive.mapred.supports.subdirectories=true;
 CREATE TABLE  if not exists stored_as_dirs_multiple (col1 STRING, col2 int, col3 STRING) 
 SKEWED BY (col1, col2) ON (('s1',1), ('s3',3), ('s13',13), ('s78',78))  stored as DIRECTORIES;
 
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE stored_as_dirs_multiple;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE stored_as_dirs_multiple;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q
index 927f02e82bf91777cdfd047b7ed1596d810334ce..64182eac8362d83d5d31bb5ea96b2cc2998e37af 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q
@@ -1,3 +1,3 @@
 DROP VIEW xxx11;
 CREATE VIEW xxx11 AS SELECT * FROM src;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE xxx11;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE xxx11;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q
index 16feeca22649a92a42d49a72e0d0f7b7925c80f7..f0c3b59d30ddf43595af3293fa1c7865e489958c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q
@@ -3,4 +3,4 @@
 
 
 CREATE TABLE load_wrong_fileformat_T1(name STRING) STORED AS SEQUENCEFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE load_wrong_fileformat_T1;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE load_wrong_fileformat_T1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q
index 7e589fbfde6207ab1430d4c1b89198fc1da9266a..4d79bbeb102c74c6b27feaf79f3230349a0dd8d8 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q
@@ -3,4 +3,4 @@
 
 
 CREATE TABLE T1(name STRING) STORED AS RCFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.seq' INTO TABLE T1;
\ No newline at end of file
+LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T1;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q
index ff5ed4e2e3107aefc1618218ce9d29a5597d3043..050c819a2f04bafb40b3f25366d8b62e5095ce45 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q
@@ -3,4 +3,4 @@
 
 
 CREATE TABLE T1(name STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.seq' INTO TABLE T1;
\ No newline at end of file
+LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T1;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q
index ffb64ed643b14670a3529902b2d3c6461e39e393..7f5ad754142ab51dd928a159f93414b2f05697eb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q
@@ -1,3 +1,3 @@
 
 CREATE TABLE loadpart1(a STRING, b STRING) PARTITIONED BY (ds STRING,ds1 STRING);
-LOAD DATA LOCAL INPATH '../data1/files/kv1.txt' INTO TABLE loadpart1 PARTITION(ds='2009-05-05');
+LOAD DATA LOCAL INPATH '../../data1/files/kv1.txt' INTO TABLE loadpart1 PARTITION(ds='2009-05-05');
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q
index 8f4b37a9d49f6a4ed1433ed568509785ba963978..ed9e21dd8a1fbe89e1bcf5b57f15fa491011410f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q
@@ -1,4 +1,4 @@
 set hive.exec.mode.local.auto=true;
 set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateLocalErrorsHook;
 
-FROM src SELECT TRANSFORM(key, value) USING 'python ../data/scripts/cat_error.py' AS (key, value);
+FROM src SELECT TRANSFORM(key, value) USING 'python ../../data/scripts/cat_error.py' AS (key, value);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q
new file mode 100644
index 0000000000000000000000000000000000000000..4966f2b9b2825f2f1d3417bbec624ba8890027c9
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q
@@ -0,0 +1,17 @@
+create database lockneg1;
+use lockneg1;
+
+create table tstsrcpart like default.srcpart;
+
+insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11')
+select key, value from default.srcpart where ds='2008-04-08' and hr='11';
+
+lock database lockneg1 shared;
+show locks database lockneg1;
+select count(1) from tstsrcpart where ds='2008-04-08' and hr='11';
+
+unlock database lockneg1;
+show locks database lockneg1;
+lock database lockneg1 exclusive;
+show locks database lockneg1;
+select count(1) from tstsrcpart where ds='2008-04-08' and hr='11';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q
new file mode 100644
index 0000000000000000000000000000000000000000..1f9ad90898dce1303bab499a2b62b93c1606a1f9
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q
@@ -0,0 +1,6 @@
+set hive.lock.numretries=0;
+
+create database lockneg4;
+
+lock database lockneg4 exclusive;
+lock database lockneg4 shared;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q
new file mode 100644
index 0000000000000000000000000000000000000000..8cbe31083b400ce24b3bf12f49af3050f081f589
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q
@@ -0,0 +1,8 @@
+set hive.lock.numretries=0;
+
+create database lockneg9;
+
+lock database lockneg9 shared;
+show locks database lockneg9;
+
+drop database lockneg9;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q
new file mode 100644
index 0000000000000000000000000000000000000000..4127a6f150a13acf103ee7e3b9a6ec6fd0a40846
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q
@@ -0,0 +1,15 @@
+set hive.lock.numretries=0;
+
+create database lockneg2;
+use lockneg2;
+
+create table tstsrcpart like default.srcpart;
+
+insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11')
+select key, value from default.srcpart where ds='2008-04-08' and hr='11';
+
+lock table tstsrcpart shared;
+show locks;
+
+lock database lockneg2 exclusive;
+show locks;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q
index ac6c4ee549d850b80badcc1165e6c72ccd5727a7..09f13f52aeadf8b84909140a4f9d27648e0bf815 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q
@@ -10,6 +10,6 @@ simple_string string)
 
 
 -- This should fail in as extended nesting levels are not enabled using the serdeproperty hive.serialization.extend.nesting.levels
-load data local inpath '../data/files/nested_complex.txt' overwrite into table nestedcomplex;
+load data local inpath '../../data/files/nested_complex.txt' overwrite into table nestedcomplex;
 
 select * from nestedcomplex sort by simple_int;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q
index 4841f9e11c84e77d375ae260d565afac0f6dc25c..6669bf62d8822e665bc826f50d33a1a268894ce0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q
@@ -2,6 +2,6 @@
 CREATE TABLE nopart_insert(a STRING, b STRING) PARTITIONED BY (ds STRING);
 
 INSERT OVERWRITE TABLE nopart_insert 
-SELECT TRANSFORM(src.key, src.value) USING '../data/scripts/error_script' AS (tkey, tvalue)
+SELECT TRANSFORM(src.key, src.value) USING '../../data/scripts/error_script' AS (tkey, tvalue)
 FROM src;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q
index 6e5ad6eb41a8b75e3034bb6e6ff1c5b1ce75ec64..966982fd5ce520831f091e20eac2b05ea55d3184 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q
@@ -1,5 +1,5 @@
 
 CREATE TABLE nopart_load(a STRING, b STRING) PARTITIONED BY (ds STRING);
 
-load data local inpath '../data/files/kv1.txt' overwrite into table nopart_load ;
+load data local inpath '../../data/files/kv1.txt' overwrite into table nopart_load ;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias3.q
deleted file mode 100644
index 6cc3e87288a89aab8a4aa99cc63a662d3c7b8d8a..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias3.q
+++ /dev/null
@@ -1,4 +0,0 @@
-CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE;
-
-FROM src
-INSERT OVERWRITE TABLE dest1 SELECT '1234', src.key, sum(src.value) WHERE src.key < 100 group by key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q
new file mode 100644
index 0000000000000000000000000000000000000000..745a7867264e321c079d8146f60d14ae186bbc29
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q
@@ -0,0 +1,3 @@
+drop table if exists parquet_char;
+
+create table parquet_char (t char(10)) stored as parquet;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q
new file mode 100644
index 0000000000000000000000000000000000000000..89d3602fd3e97026df92266e8b18cb78c93b827c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q
@@ -0,0 +1,3 @@
+drop table if exists parquet_date;
+
+create table parquet_date (t date) stored as parquet;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q
new file mode 100644
index 0000000000000000000000000000000000000000..8a4973110a51f0d8566a8d57788e7bca18fd047a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q
@@ -0,0 +1,3 @@
+drop table if exists parquet_decimal;
+
+create table parquet_decimal (t decimal(4,2)) stored as parquet;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q
new file mode 100644
index 0000000000000000000000000000000000000000..4ef36fa0efc490d8e1a674a9133af1a04cfd5ec5
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q
@@ -0,0 +1,3 @@
+drop table if exists parquet_timestamp;
+
+create table parquet_timestamp (t timestamp) stored as parquet;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q
new file mode 100644
index 0000000000000000000000000000000000000000..55825f76dc240c54ef451ceec12adee23f12b36c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q
@@ -0,0 +1,3 @@
+drop table if exists parquet_varchar;
+
+create table parquet_varchar (t varchar(10)) stored as parquet;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q
index 72b55ea25d8e994b9cb33d66475f13be6214f88b..3fdc036996563c8367c8f157f637dc5283fa5f71 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q
@@ -4,6 +4,6 @@ drop table tbl_protectmode6;
 
 create table tbl_protectmode6  (c1 string,c2 string) partitioned by (p string);
 alter table tbl_protectmode6 add partition (p='p1');
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1');
 alter table tbl_protectmode6 partition (p='p1') enable offline; 
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1');
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q
index 542367ace22e07fb57ab8361c858cd0a9bf0dd8a..ef372259ed3e36e5d74884b4d7fdb0fc974b0ba2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q
@@ -12,7 +12,7 @@ CREATE TABLE part(
     p_comment STRING
 );
 
-LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part;
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
 
 -- testAggrFuncsWithNoGBYNoPartDef
 select p_mfgr, 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q
index 95b35113e3cf0d88ec58386ce1d7547e4afa179c..58430423436b76d24250b7c620341d74fa3bef5e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q
@@ -12,7 +12,7 @@ CREATE TABLE part(
     p_comment STRING
 );
 
-LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part;
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
 
 -- testAmbiguousWindowDefn
 select p_mfgr, p_name, p_size, 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q
index 8333ddc948419dce7f26f5d38e3d146f98d16d36..a171961a683eebbed437b8745cf322c746df3565 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q
@@ -1,2 +1,3 @@
+set hive.support.quoted.identifiers=none;
 EXPLAIN
 SELECT `+++` FROM srcpart;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q
index d1aa1f1a9542d4795cbb9b13dfe8b4957b48d155..7bac1c775522bd74c809229f3e4d0a3dcc7ee2b0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q
@@ -1,2 +1,3 @@
+set hive.support.quoted.identifiers=none;
 EXPLAIN
 SELECT `.a.` FROM srcpart;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q
index 53971916e6c96232b5626f8e0c85a84290d57672..300d145508887ad1cf6f5df35b82b0a41ef2f9e9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q
@@ -1,2 +1,3 @@
+set hive.support.quoted.identifiers=none;
 EXPLAIN
 SELECT `..`, count(1) FROM srcpart GROUP BY `..`;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe1.q
deleted file mode 100644
index 6b1c09decfb35d40d7ced0970dba6cb572d85841..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe1.q
+++ /dev/null
@@ -1,3 +0,0 @@
-set hive.exec.script.allow.partial.consumption = false;
--- Tests exception in ScriptOperator.close() by passing to the operator a small amount of data
-SELECT TRANSFORM(*) USING 'true' AS a, b FROM (SELECT TRANSFORM(*) USING 'echo' AS a, b FROM src LIMIT 1) tmp;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q
index e46aed03b147fdc61740caeb65cff6437cf04960..8ca849b82d8ad1578372d08332c90b07fae1855f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q
@@ -1,7 +1,7 @@
 EXPLAIN
-SELECT TRANSFORM(src.key, src.value) USING '../data/scripts/error_script' AS (tkey, tvalue)
+SELECT TRANSFORM(src.key, src.value) USING '../../data/scripts/error_script' AS (tkey, tvalue)
 FROM src;
 
-SELECT TRANSFORM(src.key, src.value) USING '../data/scripts/error_script' AS (tkey, tvalue)
+SELECT TRANSFORM(src.key, src.value) USING '../../data/scripts/error_script' AS (tkey, tvalue)
 FROM src;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q
index a3955744221a487093b117f5eeb32f166d09340e..d523d03e906c11c77d8860ca24867f1d79ebe882 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q
@@ -16,8 +16,8 @@ WITH SERDEPROPERTIES (
 )
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH "../data/files/apache.access.log" INTO TABLE serde_regex;
-LOAD DATA LOCAL INPATH "../data/files/apache.access.2.log" INTO TABLE serde_regex;
+LOAD DATA LOCAL INPATH "../../data/files/apache.access.log" INTO TABLE serde_regex;
+LOAD DATA LOCAL INPATH "../../data/files/apache.access.2.log" INTO TABLE serde_regex;
 
 -- raise an exception 
 SELECT * FROM serde_regex ORDER BY time;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q
new file mode 100644
index 0000000000000000000000000000000000000000..579e9408b6c35c8690af89d4857d37e9f885348e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q
@@ -0,0 +1,5 @@
+-- should fail: hive.fetch.task.conversion accepts minimal or more
+desc src;
+
+set hive.conf.validation=true;
+set hive.fetch.task.conversion=true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q
index 401cc37f67dd0c2dcc22f997cfbe2335a1fd466d..1b2872d3d7ed8d9ba129f23d421d7bed0296b371 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q
@@ -6,7 +6,7 @@
 
 create table tmptable(key string, value string);
 
-set hive.stats.dbclass=dummy;
+set hive.stats.dbclass=custom;
 set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher;
 set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator;
 set hive.test.dummystats.aggregator=connect;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q
index c7e63591adde0541f31b55dee55323d608d90640..0fa9ff68203718058d3b2795d4c7f571eb9a8fef 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q
@@ -5,7 +5,7 @@
 
 create table tmptable(key string, value string);
 
-set hive.stats.dbclass=dummy;
+set hive.stats.dbclass=custom;
 set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher;
 set hive.stats.default.aggregator="";
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q
index 7fa0f55f2a4508461ca60e73755f7d135ed8e6e6..be7c4f72feb9be33f57c43c5d98b6cb8e9a539ae 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q
@@ -6,7 +6,7 @@
 
 create table tmptable(key string, value string);
 
-set hive.stats.dbclass=dummy;
+set hive.stats.dbclass=custom;
 set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher;
 set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator;
 set hive.test.dummystats.publisher=connect;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q
index f82d4b54b697bc2dfdd2fc2d9e388abe7beacdf1..652afe7c5bfba65f52adca16fad6f1ba8f116afe 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q
@@ -5,7 +5,7 @@
 
 create table tmptable(key string, value string);
 
-set hive.stats.dbclass=dummy;
+set hive.stats.dbclass=custom;
 set hive.stats.default.publisher="";
 set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q
new file mode 100644
index 0000000000000000000000000000000000000000..9013df6f938dc307c6cc629cf6971bbba0a64272
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q
@@ -0,0 +1,10 @@
+
+
+select * 
+from src b 
+where exists 
+  (select count(*) 
+  from src a 
+  where b.value = a.value  and a.key = b.key and a.value > 'val_9'
+  )
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q
new file mode 100644
index 0000000000000000000000000000000000000000..a9bc6ee6a38cb62d008740845219f4e80585cb6a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q
@@ -0,0 +1,5 @@
+
+
+select count(*) 
+from src 
+group by src.key in (select key from src s1 where s1.key > '9')
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q
new file mode 100644
index 0000000000000000000000000000000000000000..1365389cb269a936b627d75f24dfd0d0eb92a317
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q
@@ -0,0 +1,6 @@
+
+
+
+select src.key in (select key from src s1 where s1.key > '9') 
+from src
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q
new file mode 100644
index 0000000000000000000000000000000000000000..6805c5b16b0f44d1ad64a54339a7d81114b0ddd3
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q
@@ -0,0 +1,7 @@
+
+
+explain
+ select * 
+from src 
+where src.key in (select * from src s1 where s1.key > '9')
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q
new file mode 100644
index 0000000000000000000000000000000000000000..e8c41e6b17ae7f9ea048652631fbd232220cda94
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q
@@ -0,0 +1,18 @@
+
+
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+select *
+from part x 
+where x.p_name in (select y.p_name from part y where exists (select z.p_name from part z where y.p_name = z.p_name))
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q
new file mode 100644
index 0000000000000000000000000000000000000000..852b2953ff46361501048a3d78dc60f7e0c2d338
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q
@@ -0,0 +1,10 @@
+
+
+select * 
+from src b 
+where not exists 
+  (select sum(1)
+  from src a 
+  where b.value = a.value  and a.key = b.key and a.value > 'val_9'
+  )
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q
new file mode 100644
index 0000000000000000000000000000000000000000..d442f077c0707ed10ec4f3cf01ba34f1328f3827
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q
@@ -0,0 +1,6 @@
+
+
+select *
+from src
+where src.key in (select key from src where key > '9')
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q
new file mode 100644
index 0000000000000000000000000000000000000000..8ea94c5fc6d76a7eef03caa7f994b5e222efcd63
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q
@@ -0,0 +1,6 @@
+
+explain
+select * 
+from src 
+where src.key in (select key from src) in (select key from src)
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q
new file mode 100644
index 0000000000000000000000000000000000000000..99ff9ca70383502d44b78f44f778e9b78f39a9bf
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q
@@ -0,0 +1,6 @@
+
+
+select *
+from src
+where key in (select key from src)
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q
new file mode 100644
index 0000000000000000000000000000000000000000..105d3d22d9d2bc12fc6e6bfa3119e6c650948e08
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q
@@ -0,0 +1,26 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+
+-- corr and windowing 
+select p_mfgr, p_name, p_size 
+from part a 
+where a.p_size in 
+  (select first_value(p_size) over(partition by p_mfgr order by p_size) 
+   from part b 
+   where a.p_brand = b.p_brand)
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q
new file mode 100644
index 0000000000000000000000000000000000000000..c2c322178f38645d52f0027c2945ee6cfd3ee30a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q
@@ -0,0 +1,5 @@
+
+select count(*) 
+from src 
+where src.key in (select key from src s1 where s1.key > '9') or src.value is not null
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong.q
deleted file mode 100644
index 2fb5ff74cc3ce00cfb4b3147ba8de193b344389f..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong.q
+++ /dev/null
@@ -1,6 +0,0 @@
-SELECT CASE '1'
-        WHEN 1 THEN 2
-        WHEN 3 THEN 4
-        ELSE 5
-       END
-FROM src LIMIT 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q
deleted file mode 100644
index 5772dc1a95c9d5318ac0f2f33ac2195b2872dd43..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q
+++ /dev/null
@@ -1,6 +0,0 @@
-SELECT CASE 1
-        WHEN 1 THEN '2'
-        WHEN 3 THEN 4
-        ELSE 5
-       END
-FROM src LIMIT 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q
deleted file mode 100644
index 5aaf0188eb9c673c5846b92af05fb517b48aefee..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q
+++ /dev/null
@@ -1,6 +0,0 @@
-SELECT CASE 1
-        WHEN 1 THEN NULL
-        WHEN 3 THEN '2'
-        ELSE 7
-       END
-FROM src LIMIT 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q
new file mode 100644
index 0000000000000000000000000000000000000000..68050fd95cd22cff50fb3ee29ecb1320b0dfaa43
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q
@@ -0,0 +1 @@
+select default.nonexistfunc() from src;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q
new file mode 100644
index 0000000000000000000000000000000000000000..bcfa217737e33400f505c8b034bbca0e88e5dc8e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q
@@ -0,0 +1 @@
+create function lookup as 'org.apache.hadoop.hive.ql.udf.UDFFileLookup' using file '../../data/files/sales.txt';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q
new file mode 100644
index 0000000000000000000000000000000000000000..d37665dde69bcc703384262706e134785d29f241
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q
@@ -0,0 +1 @@
+create function lookup as 'org.apache.hadoop.hive.ql.udf.UDFFileLookup' using file 'nonexistent_file.txt';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q
new file mode 100644
index 0000000000000000000000000000000000000000..476dfa21a237fe505ea52cdbeaf3aa7dfeb678fc
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q
@@ -0,0 +1 @@
+create temporary function default.myfunc as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q
deleted file mode 100644
index 79fa65f63da56ed628f0905ccb7c1fa9b66c0caa..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q
+++ /dev/null
@@ -1,6 +0,0 @@
-SELECT CASE
-        WHEN 1=2 THEN '2'
-        WHEN 3=4 THEN 4
-        ELSE 5
-       END
-FROM src LIMIT 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q
deleted file mode 100644
index 8bb5fdd7ea37773f1d60b02b16d9f0fedfdaaf3b..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q
+++ /dev/null
@@ -1,6 +0,0 @@
-SELECT CASE
-        WHEN 1=2 THEN '2'
-        WHEN 3=4 THEN '5'
-        ELSE 5.3
-       END
-FROM src LIMIT 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udfnull.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udfnull.q
deleted file mode 100644
index 3c4204f7800041576eefe6234180e78141e2a43d..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udfnull.q
+++ /dev/null
@@ -1,6 +0,0 @@
-
-CREATE TEMPORARY FUNCTION example_arraysum AS 'org.apache.hadoop.hive.contrib.udf.example.UDFExampleArraySum';
-
-SELECT example_arraysum(lint)FROM src_thrift;
-
-DROP TEMPORARY FUNCTION example_arraysum;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union.q
deleted file mode 100644
index e3c5c830897eaa35595a8b3924a17b407b27d07c..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union.q
+++ /dev/null
@@ -1,4 +0,0 @@
-explain
-select s1.key as key, s1.value as value from src s1
-  UNION  ALL  
-select s2.key as key, s2.value as value from src s2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q
new file mode 100644
index 0000000000000000000000000000000000000000..c5b593e4bb556d87992aaad6c1eab280d5b41e94
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q
@@ -0,0 +1 @@
+select nonexistfunc(key) over () from src limit 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q
new file mode 100644
index 0000000000000000000000000000000000000000..15f8fae292bbb42418f5a001bbbf6bbb71eed529
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q
@@ -0,0 +1,26 @@
+DROP TABLE IF EXISTS part;
+
+-- data setup
+CREATE TABLE part(
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+
+select p_mfgr, p_name, p_size,
+min(p_retailprice),
+rank() over(distribute by p_mfgr sort by p_name)as r,
+dense_rank() over(distribute by p_mfgr sort by p_name) as dr,
+p_size, p_size - lag(p_size,-1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz
+from part
+group by p_mfgr, p_name, p_size
+;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter1.q
index 5fd19455f93cd35bec1ccf3d3b911c8509bfbeb0..312a0177fafff71edfe823b4daa04ac23a9691bd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter1.q
@@ -15,7 +15,7 @@ describe extended alter1;
 alter table alter1 set serdeproperties('s1'='10', 's2' ='20');
 describe extended alter1;
 
-add jar ../data/files/TestSerDe.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar;
 alter table alter1 set serde 'org.apache.hadoop.hive.serde2.TestSerDe' with serdeproperties('s1'='9');
 describe extended alter1;
 
@@ -56,7 +56,7 @@ DESCRIBE EXTENDED alter1;
 ALTER TABLE alter1 SET SERDEPROPERTIES('s1'='10', 's2' ='20');
 DESCRIBE EXTENDED alter1;
 
-add jar ../data/files/TestSerDe.jar;
+add jar ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar;
 ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' WITH SERDEPROPERTIES ('s1'='9');
 DESCRIBE EXTENDED alter1;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q
index 3cde00748b98ef69e19ecb8a32fb090d1dd4a598..91e4e9bad0edaf032adc36d4af8ae6a6e9052510 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q
@@ -1,5 +1,5 @@
 create table alter3_src ( col1 string ) stored as textfile ;
-load data local inpath '../data/files/test.dat' overwrite into table alter3_src ;
+load data local inpath '../../data/files/test.dat' overwrite into table alter3_src ;
 
 create table alter3 ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile;
 
@@ -34,7 +34,7 @@ USE alter3_db;
 SHOW TABLES;
 
 CREATE TABLE alter3_src (col1 STRING) STORED AS TEXTFILE ;
-LOAD DATA LOCAL INPATH '../data/files/test.dat' OVERWRITE INTO TABLE alter3_src ;
+LOAD DATA LOCAL INPATH '../../data/files/test.dat' OVERWRITE INTO TABLE alter3_src ;
 
 CREATE TABLE alter3 (col1 STRING) PARTITIONED BY (pcol1 STRING, pcol2 STRING) STORED AS SEQUENCEFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q
index 0d14f228d1c30011c091faf311b97e60c7d69d35..66c9f8dc54b9164bf6fc8d0fcac003ab9733f032 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q
@@ -3,7 +3,7 @@
 --
 
 create table alter5_src ( col1 string ) stored as textfile ;
-load data local inpath '../data/files/test.dat' overwrite into table alter5_src ;
+load data local inpath '../../data/files/test.dat' overwrite into table alter5_src ;
 
 create table alter5 ( col1 string ) partitioned by (dt string);
 
@@ -32,7 +32,7 @@ USE alter5_db;
 SHOW TABLES;
 
 create table alter5_src ( col1 string ) stored as textfile ;
-load data local inpath '../data/files/test.dat' overwrite into table alter5_src ;
+load data local inpath '../../data/files/test.dat' overwrite into table alter5_src ;
 
 create table alter5 ( col1 string ) partitioned by (dt string);
 alter table alter5 add partition (dt='a') location 'parta';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char1.q
new file mode 100644
index 0000000000000000000000000000000000000000..4ecb7e7389f6ba49d17667150021b41ce63874ab
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char1.q
@@ -0,0 +1,32 @@
+drop table alter_char_1;
+
+create table alter_char_1 (key string, value string);
+insert overwrite table alter_char_1
+  select key, value from src order by key limit 5;
+
+select * from alter_char_1 order by key;
+
+-- change column to char
+alter table alter_char_1 change column value value char(20);
+-- contents should still look the same
+select * from alter_char_1 order by key;
+
+-- change column to smaller char
+alter table alter_char_1 change column value value char(3);
+-- value column should be truncated now
+select * from alter_char_1 order by key;
+
+-- change back to bigger char
+alter table alter_char_1 change column value value char(20);
+-- column values should be full size again
+select * from alter_char_1 order by key;
+
+-- add char column
+alter table alter_char_1 add columns (key2 int, value2 char(10));
+select * from alter_char_1 order by key;
+
+insert overwrite table alter_char_1
+  select key, value, key, value from src order by key limit 5;
+select * from alter_char_1 order by key;
+
+drop table alter_char_1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char2.q
new file mode 100644
index 0000000000000000000000000000000000000000..7fa9fcef11a14fd4d558418bf8f0d2ce7394bb67
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char2.q
@@ -0,0 +1,22 @@
+
+-- alter column type, with partitioned table
+drop table if exists alter_char2;
+
+create table alter_char2 (
+  c1 char(255)
+) partitioned by (hr int);
+
+insert overwrite table alter_char2 partition (hr=1)
+  select value from src limit 1;
+
+select c1, length(c1) from alter_char2;
+
+alter table alter_char2 change column c1 c1 char(10);
+
+select hr, c1, length(c1) from alter_char2 where hr = 1;
+
+insert overwrite table alter_char2 partition (hr=2)
+  select key from src limit 1;
+
+select hr, c1, length(c1) from alter_char2 where hr = 1;
+select hr, c1, length(c1) from alter_char2 where hr = 2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q
index 807ef539c864335823bc6533141eabeab244089d..e1c3780a9b0ec5de0ab86022a3c2a4d3f64f48bd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q
@@ -1,9 +1,9 @@
 set hive.exec.concatenate.check.index =false;
 create table src_rc_concatenate_test(key int, value string) stored as rcfile;
 
-load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_concatenate_test;
-load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_concatenate_test;
-load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_concatenate_test;
+load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_concatenate_test;
+load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_concatenate_test;
+load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_concatenate_test;
 
 show table extended like `src_rc_concatenate_test`;
 
@@ -26,9 +26,9 @@ create table src_rc_concatenate_test_part(key int, value string) partitioned by
 
 alter table src_rc_concatenate_test_part add partition (ds='2011');
 
-load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_concatenate_test_part partition (ds='2011');
-load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_concatenate_test_part partition (ds='2011');
-load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_concatenate_test_part partition (ds='2011');
+load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_concatenate_test_part partition (ds='2011');
+load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_concatenate_test_part partition (ds='2011');
+load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_concatenate_test_part partition (ds='2011');
 
 show table extended like `src_rc_concatenate_test_part` partition (ds='2011');
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_db_owner.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_db_owner.q
new file mode 100644
index 0000000000000000000000000000000000000000..b224f3339e4ab489b1598b7ad269c4c308bb61c5
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_db_owner.q
@@ -0,0 +1,9 @@
+
+create database db_alter_onr;
+describe database db_alter_onr;
+
+alter database db_alter_onr set owner user user1;
+describe database db_alter_onr;
+
+alter database db_alter_onr set owner role role1;
+describe database db_alter_onr;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q
index c3502739d54a9554adecc0ee802cab89fbc4ce1b..ceabd0830a03dea1da0e3011d858913f23850f7b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q
@@ -1,8 +1,8 @@
 create table src_rc_merge_test(key int, value string) stored as rcfile;
 
-load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test;
-load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test;
-load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test;
+load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test;
+load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test;
+load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test;
 
 show table extended like `src_rc_merge_test`;
 
@@ -21,9 +21,9 @@ create table src_rc_merge_test_part(key int, value string) partitioned by (ds st
 
 alter table src_rc_merge_test_part add partition (ds='2011');
 
-load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_part partition (ds='2011');
-load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_part partition (ds='2011');
-load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_part partition (ds='2011');
+load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test_part partition (ds='2011');
+load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test_part partition (ds='2011');
+load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test_part partition (ds='2011');
 
 show table extended like `src_rc_merge_test_part` partition (ds='2011');
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q
index 65ddfed13dac474be1324f12c8c5f3e294422e45..e09703d1c6c8529ff3dd92fa322906faff11cf97 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q
@@ -3,9 +3,9 @@ create table src_rc_merge_test_part(key int, value string) partitioned by (ds st
 alter table src_rc_merge_test_part add partition (ds='2012-01-03', ts='2012-01-03+14:46:31');
 desc extended src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31');
 
-load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31');
-load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31');
-load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31');
+load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31');
+load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31');
+load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31');
 
 select count(1) from src_rc_merge_test_part where ds='2012-01-03' and ts='2012-01-03+14:46:31';
 select sum(hash(key)), sum(hash(value)) from src_rc_merge_test_part where ds='2012-01-03' and ts='2012-01-03+14:46:31';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q
index 23bae55ab2add17cfd4567fc82e71ef2eb0b761c..0af87e2c076e641b11ffd4432e17075bcbfdc65a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q
@@ -1,8 +1,8 @@
 create table src_rc_merge_test_stat(key int, value string) stored as rcfile;
 
-load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_stat;
-load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_stat;
-load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_stat;
+load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test_stat;
+load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test_stat;
+load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test_stat;
 
 show table extended like `src_rc_merge_test_stat`;
 desc extended src_rc_merge_test_stat;
@@ -21,9 +21,9 @@ create table src_rc_merge_test_part_stat(key int, value string) partitioned by (
 
 alter table src_rc_merge_test_part_stat add partition (ds='2011');
 
-load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_part_stat partition (ds='2011');
-load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_part_stat partition (ds='2011');
-load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_part_stat partition (ds='2011');
+load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test_part_stat partition (ds='2011');
+load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test_part_stat partition (ds='2011');
+load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test_part_stat partition (ds='2011');
 
 show table extended like `src_rc_merge_test_part_stat` partition (ds='2011');
 desc extended src_rc_merge_test_part_stat;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q
index b6d1eb8f2d4b8e38a508ab4c599b7b802cf19e9b..5dda4c08fd5b3eda42a916fb22783f8cb0a3a945 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q
@@ -1,4 +1,4 @@
-
+-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S)
 create table tst1(key string, value string) partitioned by (ds string) clustered by (key) into 10 buckets;
 
 alter table tst1 clustered by (key) into 8 buckets;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q
index c6a4ad24fcdde57dfcc869cac5360f6a4e1558cd..acc028bdd81eb36294f3291717fdf1daf4f45c3f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q
@@ -1,3 +1,4 @@
+-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S)
 -- Tests that when overwriting a partition in a table after altering the bucketing/sorting metadata
 -- the partition metadata is updated as well.
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2_h23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2_h23.q
new file mode 100644
index 0000000000000000000000000000000000000000..d81430441c720ae077cbfd764c7f43df59708b6b
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2_h23.q
@@ -0,0 +1,85 @@
+-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S)
+-- Tests that when overwriting a partition in a table after altering the bucketing/sorting metadata
+-- the partition metadata is updated as well.
+
+CREATE TABLE tst1(key STRING, value STRING) PARTITIONED BY (ds STRING);
+
+DESCRIBE FORMATTED tst1;
+
+SET hive.enforce.bucketing=true;
+SET hive.enforce.sorting=true;
+INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src;
+
+DESCRIBE FORMATTED tst1 PARTITION (ds = '1');
+
+-- Test an unbucketed partition gets converted to bucketed
+ALTER TABLE tst1 CLUSTERED BY (key) INTO 8 BUCKETS;
+
+DESCRIBE FORMATTED tst1;
+
+INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src;
+
+DESCRIBE FORMATTED tst1 PARTITION (ds = '1');
+
+-- Test an unsorted partition gets converted to sorted
+ALTER TABLE tst1 CLUSTERED BY (key) SORTED BY (key DESC) INTO 8 BUCKETS;
+
+DESCRIBE FORMATTED tst1;
+
+INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src;
+
+DESCRIBE FORMATTED tst1 PARTITION (ds = '1');
+
+-- Test changing the bucket columns
+ALTER TABLE tst1 CLUSTERED BY (value) SORTED BY (key DESC) INTO 8 BUCKETS;
+
+DESCRIBE FORMATTED tst1;
+
+INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src;
+
+DESCRIBE FORMATTED tst1 PARTITION (ds = '1');
+
+-- Test changing the number of buckets
+ALTER TABLE tst1 CLUSTERED BY (value) SORTED BY (key DESC) INTO 4 BUCKETS;
+
+DESCRIBE FORMATTED tst1;
+
+INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src;
+
+DESCRIBE FORMATTED tst1 PARTITION (ds = '1');
+
+-- Test changing the sort columns
+ALTER TABLE tst1 CLUSTERED BY (value) SORTED BY (value DESC) INTO 4 BUCKETS;
+
+DESCRIBE FORMATTED tst1;
+
+INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src;
+
+DESCRIBE FORMATTED tst1 PARTITION (ds = '1');
+
+-- Test changing the sort order
+ALTER TABLE tst1 CLUSTERED BY (value) SORTED BY (value ASC) INTO 4 BUCKETS;
+
+DESCRIBE FORMATTED tst1;
+
+INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src;
+
+DESCRIBE FORMATTED tst1 PARTITION (ds = '1');
+
+-- Test a sorted partition gets converted to unsorted
+ALTER TABLE tst1 CLUSTERED BY (value) INTO 4 BUCKETS;
+
+DESCRIBE FORMATTED tst1;
+
+INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src;
+
+DESCRIBE FORMATTED tst1 PARTITION (ds = '1');
+
+-- Test a bucketed partition gets converted to unbucketed
+ALTER TABLE tst1 NOT CLUSTERED;
+
+DESCRIBE FORMATTED tst1;
+
+INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src;
+
+DESCRIBE FORMATTED tst1 PARTITION (ds = '1');
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table_h23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table_h23.q
new file mode 100644
index 0000000000000000000000000000000000000000..a03992510b0722b84faee372994f42ee499c4fc0
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table_h23.q
@@ -0,0 +1,59 @@
+-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S)
+create table tst1(key string, value string) partitioned by (ds string) clustered by (key) into 10 buckets;
+
+alter table tst1 clustered by (key) into 8 buckets;
+
+describe formatted tst1;
+
+set hive.enforce.bucketing=true;
+insert overwrite table tst1 partition (ds='1') select key, value from src;
+
+describe formatted tst1 partition (ds = '1');
+
+-- Test changing bucket number
+
+alter table tst1 clustered by (key) into 12 buckets;
+
+insert overwrite table tst1 partition (ds='1') select key, value from src;
+
+describe formatted tst1 partition (ds = '1');
+
+describe formatted tst1;
+
+-- Test changing bucket number of (table/partition)
+
+alter table tst1 into 4 buckets;
+
+describe formatted tst1;
+
+describe formatted tst1 partition (ds = '1');
+
+alter table tst1 partition (ds = '1') into 6 buckets;
+
+describe formatted tst1;
+
+describe formatted tst1 partition (ds = '1');
+
+-- Test adding sort order
+
+alter table tst1 clustered by (key) sorted by (key asc) into 12 buckets;
+
+describe formatted tst1;
+
+-- Test changing sort order
+
+alter table tst1 clustered by (key) sorted by (value desc) into 12 buckets;
+
+describe formatted tst1;
+
+-- Test removing test order
+
+alter table tst1 clustered by (value) into 12 buckets;
+
+describe formatted tst1;
+
+-- Test removing buckets
+
+alter table tst1 not clustered;
+
+describe formatted tst1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q
index 5479afbbd5ab0e3997008033406a917b87190b70..19c0f9d1d88ec9f6aa1aa94ca58874ecbad87551 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q
@@ -10,48 +10,48 @@ desc alter_coltype;
 select count(*) from alter_coltype where dt = '100x';
 
 -- alter partition key column data type for dt column.
-alter table alter_coltype partition column (dt int);
+-- alter table alter_coltype partition column (dt int);
 
 -- load a new partition using new data type.
-insert overwrite table alter_coltype partition(dt=10, ts='3.0') select * from src1;
+-- insert overwrite table alter_coltype partition(dt=10, ts='3.0') select * from src1;
 
 -- make sure the partition predicate still works. 
-select count(*) from alter_coltype where dt = '100x';
-explain extended select count(*) from alter_coltype where dt = '100x';
+-- select count(*) from alter_coltype where dt = '100x';
+-- explain extended select count(*) from alter_coltype where dt = '100x';
 
-select count(*) from alter_coltype where dt = 100;
+-- select count(*) from alter_coltype where dt = '100';
 
 -- alter partition key column data type for ts column.
-alter table alter_coltype partition column (ts double);
+-- alter table alter_coltype partition column (ts double);
 
-alter table alter_coltype partition column (dt string);
+-- alter table alter_coltype partition column (dt string);
 
 -- load a new partition using new data type.
-insert overwrite table alter_coltype partition(dt='100x', ts=3.0) select * from src1;
+-- insert overwrite table alter_coltype partition(dt='100x', ts=3.0) select * from src1;
 
 --  validate partition key column predicate can still work.
-select count(*) from alter_coltype where ts = '6:30pm';
-explain extended select count(*) from alter_coltype where ts = '6:30pm';
+-- select count(*) from alter_coltype where ts = '6:30pm';
+-- explain extended select count(*) from alter_coltype where ts = '6:30pm';
 
 --  validate partition key column predicate on two different partition column data type 
 --  can still work.
-select count(*) from alter_coltype where ts = 3.0 and dt=10;
-explain extended select count(*) from alter_coltype where ts = 3.0 and dt=10;
+-- select count(*) from alter_coltype where ts = 3.0 and dt=10;
+-- explain extended select count(*) from alter_coltype where ts = 3.0 and dt=10;
 
 -- query where multiple partition values (of different datatypes) are being selected 
-select key, value, dt, ts from alter_coltype where dt is not null;
-explain extended select key, value, dt, ts from alter_coltype where dt is not null;
+-- select key, value, dt, ts from alter_coltype where dt is not null;
+-- explain extended select key, value, dt, ts from alter_coltype where dt is not null;
 
-select count(*) from alter_coltype where ts = 3.0;
+-- select count(*) from alter_coltype where ts = 3.0;
 
 -- make sure the partition predicate still works. 
-select count(*) from alter_coltype where dt = '100x' or dt = '10';
-explain extended select count(*) from alter_coltype where dt = '100x' or dt = '10';
+-- select count(*) from alter_coltype where dt = '100x' or dt = '10';
+-- explain extended select count(*) from alter_coltype where dt = '100x' or dt = '10';
 
-desc alter_coltype;
-desc alter_coltype partition (dt='100x', ts='6:30pm');
-desc alter_coltype partition (dt='100x', ts=3.0);
-desc alter_coltype partition (dt=10, ts=3.0);
+-- desc alter_coltype;
+-- desc alter_coltype partition (dt='100x', ts='6:30pm');
+-- desc alter_coltype partition (dt='100x', ts=3.0);
+-- desc alter_coltype partition (dt=10, ts=3.0);
 
 drop table alter_coltype;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q
index 7bcb9f071c415841d8c309468d56fc3912eb4ecd..7a1f3dd51d61705ffe97916d0ea8691430a3b753 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q
@@ -2,10 +2,10 @@
 create table if not exists alter_part_protect_mode(key string, value string ) partitioned by (year string, month string) stored as textfile ;
 
 -- Load data
-load data local inpath '../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1996', month='10');
-load data local inpath '../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1996', month='12');
-load data local inpath '../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1995', month='09');
-load data local inpath '../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1994', month='07');
+load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1996', month='10');
+load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1996', month='12');
+load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1995', month='09');
+load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1994', month='07');
 
 -- offline
 alter table alter_part_protect_mode partition (year='1996') disable offline;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q
index d498cd52a5ffb5a9ee8071fddc08ab3d2008b774..8ebbe98824dec34c0d7274a01ffd7037ad3d8816 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q
@@ -4,7 +4,7 @@ DROP TABLE alter_rename_partition;
 SHOW TABLES;
 
 create table alter_rename_partition_src ( col1 string ) stored as textfile ;
-load data local inpath '../data/files/test.dat' overwrite into table alter_rename_partition_src ;
+load data local inpath '../../data/files/test.dat' overwrite into table alter_rename_partition_src ;
 
 create table alter_rename_partition ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile;
 
@@ -28,7 +28,7 @@ USE alter_rename_partition_db;
 SHOW TABLES;
 
 CREATE TABLE alter_rename_partition_src (col1 STRING) STORED AS TEXTFILE ;
-LOAD DATA LOCAL INPATH '../data/files/test.dat' OVERWRITE INTO TABLE alter_rename_partition_src ;
+LOAD DATA LOCAL INPATH '../../data/files/test.dat' OVERWRITE INTO TABLE alter_rename_partition_src ;
 
 CREATE TABLE alter_rename_partition (col1 STRING) PARTITIONED BY (pcol1 STRING, pcol2 STRING) STORED AS SEQUENCEFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q
index 5a481e7f8ebfe9918ce7364f30d28a2090ec4adc..b870108bddd2b92e932edf5be287444820285940 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q
@@ -7,7 +7,7 @@ create table alter_varchar2 (
 ) partitioned by (hr int);
 
 insert overwrite table alter_varchar2 partition (hr=1)
-  select value from src limit 1;
+  select value from src tablesample (1 rows);
 
 select c1, length(c1) from alter_varchar2;
 
@@ -16,7 +16,9 @@ alter table alter_varchar2 change column c1 c1 varchar(10);
 select hr, c1, length(c1) from alter_varchar2 where hr = 1;
 
 insert overwrite table alter_varchar2 partition (hr=2)
-  select key from src limit 1;
+  select key from src tablesample (1 rows);
+
+set hive.fetch.task.conversion=more;
 
 select hr, c1, length(c1) from alter_varchar2 where hr = 1;
 select hr, c1, length(c1) from alter_varchar2 where hr = 2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q
index e7053c1c8eb5574a89c27307ba0083c4fd39ebdd..5ccd2c8c62dcf2784e2ceff5a607f601fb1d60fd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q
@@ -1,3 +1,4 @@
+set hive.support.quoted.identifiers=none;
 -- TOK_ALLCOLREF
 explain select * from (select a.key, a.* from (select * from src) a join (select * from src1) b on (a.key = b.key)) t;
 -- DOT
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_filter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_filter.q
new file mode 100644
index 0000000000000000000000000000000000000000..ec973e15969d4957ef699219e81bb3019c6386a1
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_filter.q
@@ -0,0 +1,76 @@
+set hive.stats.fetch.column.stats=true;
+
+create table if not exists loc_staging (
+  state string,
+  locid int,
+  zip bigint,
+  year int
+) row format delimited fields terminated by '|' stored as textfile;
+
+create table loc_orc like loc_staging;
+alter table loc_orc set fileformat orc;
+
+load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging;
+
+insert overwrite table loc_orc select * from loc_staging;
+
+-- numRows: 8 rawDataSize: 796
+explain extended select * from loc_orc;
+
+-- column stats are not COMPLETE, so stats are not updated
+-- numRows: 8 rawDataSize: 796
+explain extended select * from loc_orc where state='OH';
+
+analyze table loc_orc compute statistics for columns state,locid,zip,year;
+
+-- state column has 5 distincts. numRows/countDistincts
+-- numRows: 1 rawDataSize: 102
+explain extended select * from loc_orc where state='OH';
+
+-- not equals comparison shouldn't affect number of rows
+-- numRows: 8 rawDataSize: 804
+explain extended select * from loc_orc where state!='OH';
+explain extended select * from loc_orc where state<>'OH';
+
+-- nulls are treated as constant equality comparison
+-- numRows: 1 rawDataSize: 102
+explain extended select * from loc_orc where zip is null;
+-- numRows: 1 rawDataSize: 102
+explain extended select * from loc_orc where !(zip is not null);
+
+-- not nulls are treated as inverse of nulls
+-- numRows: 7 rawDataSize: 702
+explain extended select * from loc_orc where zip is not null;
+-- numRows: 7 rawDataSize: 702
+explain extended select * from loc_orc where !(zip is null);
+
+-- NOT evaluation. true will pass all rows, false will not pass any rows
+-- numRows: 8 rawDataSize: 804
+explain extended select * from loc_orc where !false;
+-- numRows: 0 rawDataSize: 0
+explain extended select * from loc_orc where !true;
+
+-- OR evaluation. 1 row for OH and 1 row for CA
+-- numRows: 2 rawDataSize: 204
+explain extended select * from loc_orc where state='OH' or state='CA';
+
+-- AND evaluation. cascadingly apply rules. 8/2 = 4/2 = 2
+-- numRows: 2 rawDataSize: 204
+explain extended select * from loc_orc where year=2001 and year is null;
+-- numRows: 1 rawDataSize: 102
+explain extended select * from loc_orc where year=2001 and state='OH' and state='FL';
+
+-- AND and OR together. left expr will yield 1 row and right will yield 1 row
+-- numRows: 3 rawDataSize: 306
+explain extended select * from loc_orc where (year=2001 and year is null) or (state='CA');
+
+-- AND and OR together. left expr will yield 8 rows and right will yield 1 row
+-- numRows: 1 rawDataSize: 102
+explain extended select * from loc_orc where (year=2001 or year is null) and (state='CA');
+
+-- all inequality conditions rows/3 is the rules
+-- numRows: 2 rawDataSize: 204
+explain extended select * from loc_orc where locid < 30;
+explain extended select * from loc_orc where locid > 30;
+explain extended select * from loc_orc where locid <= 30;
+explain extended select * from loc_orc where locid >= 30;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_groupby.q
new file mode 100644
index 0000000000000000000000000000000000000000..05cb036b466a2f5b89f47b8c5240728dc4d1b2f1
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_groupby.q
@@ -0,0 +1,69 @@
+set hive.stats.fetch.column.stats=true;
+
+create table if not exists loc_staging (
+  state string,
+  locid int,
+  zip bigint,
+  year int
+) row format delimited fields terminated by '|' stored as textfile;
+
+create table loc_orc like loc_staging;
+alter table loc_orc set fileformat orc;
+
+load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging;
+
+insert overwrite table loc_orc select * from loc_staging;
+
+-- numRows: 8 rawDataSize: 796
+explain extended select * from loc_orc;
+
+-- partial column stats
+analyze table loc_orc compute statistics for columns state;
+
+-- inner group by: map - numRows: 8 reduce - numRows: 4
+-- outer group by: map - numRows: 4 reduce numRows: 2
+explain extended select a, c, min(b)
+from ( select state as a, locid as b, count(*) as c
+       from loc_orc
+       group by state,locid
+     ) sq1
+group by a,c;
+
+analyze table loc_orc compute statistics for columns state,locid,zip,year;
+
+-- only one distinct value in year column + 1 NULL value
+-- map-side GBY: numRows: 8 (map-side will not do any reduction)
+-- reduce-side GBY: numRows: 2
+explain extended select year from loc_orc group by year;
+
+-- map-side GBY: numRows: 8
+-- reduce-side GBY: numRows: 4
+explain extended select state,locid from loc_orc group by state,locid;
+
+-- map-side GBY numRows: 32 reduce-side GBY numRows: 16
+explain extended select state,locid from loc_orc group by state,locid with cube;
+
+-- map-side GBY numRows: 24 reduce-side GBY numRows: 12
+explain extended select state,locid from loc_orc group by state,locid with rollup;
+
+-- map-side GBY numRows: 8 reduce-side GBY numRows: 4
+explain extended select state,locid from loc_orc group by state,locid grouping sets((state));
+
+-- map-side GBY numRows: 16 reduce-side GBY numRows: 8
+explain extended select state,locid from loc_orc group by state,locid grouping sets((state),(locid));
+
+-- map-side GBY numRows: 24 reduce-side GBY numRows: 12
+explain extended select state,locid from loc_orc group by state,locid grouping sets((state),(locid),());
+
+-- map-side GBY numRows: 32 reduce-side GBY numRows: 16
+explain extended select state,locid from loc_orc group by state,locid grouping sets((state,locid),(state),(locid),());
+
+set hive.stats.map.parallelism=10;
+
+-- map-side GBY: numRows: 80 (map-side will not do any reduction)
+-- reduce-side GBY: numRows: 2 Reason: numDistinct of year is 2. numRows = min(80/2, 2)
+explain extended select year from loc_orc group by year;
+
+-- map-side GBY numRows: 320 reduce-side GBY numRows: 42 Reason: numDistinct of state and locid are 6,7 resp. numRows = min(320/2, 6*7)
+explain extended select state,locid from loc_orc group by state,locid with cube;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q
new file mode 100644
index 0000000000000000000000000000000000000000..965b0b7ed0a3ef2b14a27fb776fb039af01a2e85
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q
@@ -0,0 +1,81 @@
+set hive.stats.fetch.column.stats=true;
+
+create table if not exists emp_staging (
+  lastname string,
+  deptid int
+) row format delimited fields terminated by '|' stored as textfile;
+
+create table if not exists dept_staging (
+  deptid int,
+  deptname string
+) row format delimited fields terminated by '|' stored as textfile;
+
+create table if not exists loc_staging (
+  state string,
+  locid int,
+  zip bigint,
+  year int
+) row format delimited fields terminated by '|' stored as textfile;
+
+create table if not exists emp_orc like emp_staging;
+alter table emp_orc set fileformat orc;
+
+create table if not exists dept_orc like dept_staging;
+alter table dept_orc set fileformat orc;
+
+create table loc_orc like loc_staging;
+alter table loc_orc set fileformat orc;
+
+LOAD DATA LOCAL INPATH '../../data/files/emp.txt' OVERWRITE INTO TABLE emp_staging;
+LOAD DATA LOCAL INPATH '../../data/files/dept.txt' OVERWRITE INTO TABLE dept_staging;
+LOAD DATA LOCAL INPATH '../../data/files/loc.txt' OVERWRITE INTO TABLE loc_staging;
+
+insert overwrite table emp_orc select * from emp_staging;
+insert overwrite table dept_orc select * from dept_staging;
+insert overwrite table loc_orc select * from loc_staging;
+
+analyze table emp_orc compute statistics for columns lastname,deptid;
+analyze table dept_orc compute statistics for columns deptname,deptid;
+analyze table loc_orc compute statistics for columns state,locid,zip,year;
+
+-- number of rows
+-- emp_orc  - 6
+-- dept_orc - 4
+-- loc_orc  - 8
+
+-- count distincts for relevant columns (since count distinct values are approximate in some cases count distint values will be greater than number of rows)
+-- emp_orc.deptid - 3
+-- emp_orc.lastname - 7
+-- dept_orc.deptid - 6
+-- dept_orc.deptname - 5
+-- loc_orc.locid - 6
+-- loc_orc.state - 7
+
+-- Expected output rows: 4
+-- Reason: #rows = (6*4)/max(3,6)
+explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid);
+
+-- 3 way join
+-- Expected output rows: 4
+-- Reason: #rows = (6*4*6)/max(3,6)*max(6,3)
+explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid) join emp_orc e1 on (e.deptid = e1.deptid);
+
+-- Expected output rows: 5
+-- Reason: #rows = (6*4*8)/max(3,6)*max(6,6)
+explain extended select * from emp_orc e join dept_orc d  on (e.deptid = d.deptid) join loc_orc l on (e.deptid = l.locid);
+
+-- join keys of different types
+-- Expected output rows: 4
+-- Reason: #rows = (6*4*8)/max(3,6)*max(6,7)
+explain extended select * from emp_orc e join dept_orc d  on (e.deptid = d.deptid) join loc_orc l on (e.deptid = l.state);
+
+-- multi-attribute join
+-- Expected output rows: 0
+-- Reason: #rows = (6*4)/max(3,6)*max(7,5)
+explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid and e.lastname = d.deptname);
+
+-- 3 way and multi-attribute join
+-- Expected output rows: 0
+-- Reason: #rows = (6*4*8)/max(3,6)*max(7,5)*max(3,6)*max(7,7)
+explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid and e.lastname = d.deptname) join loc_orc l on (e.deptid = l.locid and e.lastname = l.state);
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_limit.q
new file mode 100644
index 0000000000000000000000000000000000000000..0a9f880b5f31ed3dc0d07e213f015ff8bf2bf9d5
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_limit.q
@@ -0,0 +1,30 @@
+set hive.stats.fetch.column.stats=true;
+
+create table if not exists loc_staging (
+  state string,
+  locid int,
+  zip bigint,
+  year int
+) row format delimited fields terminated by '|' stored as textfile;
+
+create table loc_orc like loc_staging;
+alter table loc_orc set fileformat orc;
+
+load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging;
+
+insert overwrite table loc_orc select * from loc_staging;
+
+analyze table loc_orc compute statistics for columns state, locid, zip, year;
+
+-- numRows: 8 rawDataSize: 796
+explain extended select * from loc_orc;
+
+-- numRows: 4 rawDataSize: 396
+explain extended select * from loc_orc limit 4;
+
+-- greater than the available number of rows
+-- numRows: 8 rawDataSize: 796
+explain extended select * from loc_orc limit 16;
+
+-- numRows: 0 rawDataSize: 0
+explain extended select * from loc_orc limit 0;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_part.q
new file mode 100644
index 0000000000000000000000000000000000000000..839c7d84962fc3ac33344f5a332361269075ba7a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_part.q
@@ -0,0 +1,85 @@
+set hive.stats.fetch.column.stats=true;
+set hive.stats.autogather=false;
+set hive.exec.dynamic.partition=true;
+set hive.exec.dynamic.partition.mode=nonstrict;
+
+create table if not exists loc_staging (
+  state string,
+  locid int,
+  zip bigint,
+  year string
+) row format delimited fields terminated by '|' stored as textfile;
+
+LOAD DATA LOCAL INPATH '../../data/files/loc.txt' OVERWRITE INTO TABLE loc_staging;
+
+create table if not exists loc_orc (
+  state string,
+  locid int,
+  zip bigint
+) partitioned by(year string) stored as orc;
+
+-- basicStatState: NONE colStatState: NONE
+explain extended select * from loc_orc;
+
+insert overwrite table loc_orc partition(year) select * from loc_staging;
+
+-- stats are disabled. basic stats will report the file size but not raw data size. so initial statistics will be PARTIAL
+
+-- basicStatState: PARTIAL colStatState: NONE
+explain extended select * from loc_orc;
+
+-- partition level analyze statistics for specific parition
+analyze table loc_orc partition(year='2001') compute statistics;
+
+-- basicStatState: PARTIAL colStatState: NONE
+explain extended select * from loc_orc where year='__HIVE_DEFAULT_PARTITION__';
+
+-- basicStatState: PARTIAL colStatState: NONE
+explain extended select * from loc_orc;
+
+-- basicStatState: COMPLETE colStatState: NONE
+explain extended select * from loc_orc where year='2001';
+
+-- partition level analyze statistics for all partitions
+analyze table loc_orc partition(year) compute statistics;
+
+-- basicStatState: COMPLETE colStatState: NONE
+explain extended select * from loc_orc where year='__HIVE_DEFAULT_PARTITION__';
+
+-- basicStatState: COMPLETE colStatState: NONE
+explain extended select * from loc_orc;
+
+-- basicStatState: COMPLETE colStatState: NONE
+explain extended select * from loc_orc where year='2001' or year='__HIVE_DEFAULT_PARTITION__';
+
+-- both partitions will be pruned
+-- basicStatState: NONE colStatState: NONE
+explain extended select * from loc_orc where year='2001' and year='__HIVE_DEFAULT_PARTITION__';
+
+-- partition level partial column statistics
+analyze table loc_orc partition(year='2001') compute statistics for columns state,locid;
+
+-- basicStatState: COMPLETE colStatState: NONE
+explain extended select zip from loc_orc;
+
+-- basicStatState: COMPLETE colStatState: PARTIAL
+explain extended select state from loc_orc;
+
+-- column statistics for __HIVE_DEFAULT_PARTITION__ is not supported yet. Hence colStatState reports PARTIAL
+-- basicStatState: COMPLETE colStatState: PARTIAL
+explain extended select state,locid from loc_orc;
+
+-- basicStatState: COMPLETE colStatState: COMPLETE
+explain extended select state,locid from loc_orc where year='2001';
+
+-- basicStatState: COMPLETE colStatState: NONE
+explain extended select state,locid from loc_orc where year!='2001';
+
+-- basicStatState: COMPLETE colStatState: PARTIAL
+explain extended select * from loc_orc;
+
+-- This is to test filter expression evaluation on partition column
+-- numRows: 2 dataSize: 8 basicStatState: COMPLETE colStatState: COMPLETE
+explain extended select locid from loc_orc where locid>0 and year='2001';
+explain extended select locid,year from loc_orc where locid>0 and year='2001';
+explain extended select * from (select locid,year from loc_orc) test where locid>0 and year='2001';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_select.q
new file mode 100644
index 0000000000000000000000000000000000000000..5fc3f64b90180e6663aa187435fe5b3d59f80032
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_select.q
@@ -0,0 +1,143 @@
+set hive.stats.fetch.column.stats=true;
+
+create table if not exists alltypes (
+ bo1 boolean,
+ ti1 tinyint,
+ si1 smallint,
+ i1 int,
+ bi1 bigint,
+ f1 float,
+ d1 double,
+ de1 decimal,
+ ts1 timestamp,
+ da1 timestamp,
+ s1 string,
+ vc1 varchar(5),
+ m1 map<string, string>,
+ l1 array<int>,
+ st1 struct<c1:int, c2:string>
+) row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+
+create table alltypes_orc like alltypes;
+alter table alltypes_orc set fileformat orc;
+
+load data local inpath '../../data/files/alltypes.txt' overwrite into table alltypes;
+
+insert overwrite table alltypes_orc select * from alltypes;
+
+-- basicStatState: COMPLETE colStatState: NONE numRows: 2 rawDataSize: 1514
+explain extended select * from alltypes_orc;
+
+-- statistics for complex types are not supported yet
+analyze table alltypes_orc compute statistics for columns bo1, ti1, si1, i1, bi1, f1, d1, s1, vc1;
+
+-- numRows: 2 rawDataSize: 1514
+explain extended select * from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 8
+explain extended select bo1 from alltypes_orc;
+
+-- col alias renaming
+-- numRows: 2 rawDataSize: 8
+explain extended select i1 as int1 from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 174
+explain extended select s1 from alltypes_orc;
+
+-- column statistics for complex types unsupported and so statistics will not be updated
+-- numRows: 2 rawDataSize: 1514
+explain extended select m1 from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 246
+explain extended select bo1, ti1, si1, i1, bi1, f1, d1,s1 from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 0
+explain extended select null from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 8
+explain extended select 11 from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 16
+explain extended select 11L from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 16
+explain extended select 11.0 from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 178
+explain extended select "hello" from alltypes_orc;
+explain extended select cast("hello" as char(5)) from alltypes_orc;
+explain extended select cast("hello" as varchar(5)) from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 96
+explain extended select unbase64("0xe23") from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 16
+explain extended select cast("1" as TINYINT), cast("20" as SMALLINT) from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 80
+explain extended select cast("1970-12-31 15:59:58.174" as TIMESTAMP) from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 112
+explain extended select cast("1970-12-31 15:59:58.174" as DATE) from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 224
+explain extended select cast("58.174" as DECIMAL) from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 112
+explain extended select array(1,2,3) from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 1508
+explain extended select str_to_map("a=1 b=2 c=3", " ", "=") from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 112
+explain extended select NAMED_STRUCT("a", 11, "b", 11) from alltypes_orc;
+
+-- numRows: 2 rawDataSize: 250
+explain extended select CREATE_UNION(0, "hello") from alltypes_orc;
+
+-- COUNT(*) is projected as new column. It is not projected as GenericUDF and so datasize estimate will be based on number of rows
+-- numRows: 1 rawDataSize: 8
+explain extended select count(*) from alltypes_orc;
+
+-- COUNT(1) is projected as new column. It is not projected as GenericUDF and so datasize estimate will be based on number of rows
+-- numRows: 1 rawDataSize: 8
+explain extended select count(1) from alltypes_orc;
+
+-- column statistics for complex column types will be missing. data size will be calculated from available column statistics
+-- numRows: 2 rawDataSize: 254
+explain extended select *,11 from alltypes_orc;
+
+-- subquery selects
+-- inner select - numRows: 2 rawDataSize: 8
+-- outer select - numRows: 2 rawDataSize: 8
+explain extended select i1 from (select i1 from alltypes_orc limit 10) temp;
+
+-- inner select - numRows: 2 rawDataSize: 16
+-- outer select - numRows: 2 rawDataSize: 8
+explain extended select i1 from (select i1,11 from alltypes_orc limit 10) temp;
+
+-- inner select - numRows: 2 rawDataSize: 16
+-- outer select - numRows: 2 rawDataSize: 186
+explain extended select i1,"hello" from (select i1,11 from alltypes_orc limit 10) temp;
+
+-- inner select - numRows: 2 rawDataSize: 24
+-- outer select - numRows: 2 rawDataSize: 16
+explain extended select x from (select i1,11.0 as x from alltypes_orc limit 10) temp;
+
+-- inner select - numRows: 2 rawDataSize: 104
+-- outer select - numRows: 2 rawDataSize: 186
+explain extended select x,"hello" from (select i1 as x, unbase64("0xe23") as ub from alltypes_orc limit 10) temp;
+
+-- inner select -  numRows: 2 rawDataSize: 186
+-- middle select - numRows: 2 rawDataSize: 178
+-- outer select -  numRows: 2 rawDataSize: 194
+explain extended select h, 11.0 from (select hell as h from (select i1, "hello" as hell from alltypes_orc limit 10) in1 limit 10) in2;
+
+-- This test is for FILTER operator where filter expression is a boolean column
+-- numRows: 2 rawDataSize: 8
+explain extended select bo1 from alltypes_orc where bo1;
+
+-- numRows: 0 rawDataSize: 0
+explain extended select bo1 from alltypes_orc where !bo1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_table.q
new file mode 100644
index 0000000000000000000000000000000000000000..4140fe610d7cd15111978b543cacb58357954ff8
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_table.q
@@ -0,0 +1,53 @@
+set hive.stats.fetch.column.stats=true;
+set hive.stats.autogather=false;
+
+create table if not exists emp_staging (
+  lastname string,
+  deptid int
+) row format delimited fields terminated by '|' stored as textfile;
+
+create table if not exists emp_orc like emp_staging;
+alter table emp_orc set fileformat orc;
+
+-- basicStatState: NONE colStatState: NONE
+explain extended select * from emp_orc;
+
+LOAD DATA LOCAL INPATH '../../data/files/emp.txt' OVERWRITE INTO TABLE emp_staging;
+
+insert overwrite table emp_orc select * from emp_staging;
+
+-- stats are disabled. basic stats will report the file size but not raw data size. so initial statistics will be PARTIAL
+
+-- basicStatState: PARTIAL colStatState: NONE
+explain extended select * from emp_orc;
+
+-- table level analyze statistics
+analyze table emp_orc compute statistics;
+
+-- basicStatState: COMPLETE colStatState: NONE
+explain extended select * from emp_orc;
+
+-- column level partial statistics
+analyze table emp_orc compute statistics for columns deptid;
+
+-- basicStatState: COMPLETE colStatState: PARTIAL
+explain extended select * from emp_orc;
+
+-- all selected columns have statistics
+-- basicStatState: COMPLETE colStatState: COMPLETE
+explain extended select deptid from emp_orc;
+
+-- column level complete statistics
+analyze table emp_orc compute statistics for columns lastname,deptid;
+
+-- basicStatState: COMPLETE colStatState: COMPLETE
+explain extended select * from emp_orc;
+
+-- basicStatState: COMPLETE colStatState: COMPLETE
+explain extended select lastname from emp_orc;
+
+-- basicStatState: COMPLETE colStatState: COMPLETE
+explain extended select deptid from emp_orc;
+
+-- basicStatState: COMPLETE colStatState: COMPLETE
+explain extended select lastname,deptid from emp_orc;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_union.q
new file mode 100644
index 0000000000000000000000000000000000000000..586d9e1e2c023c6fedb60db5e9bab77cf9923554
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_union.q
@@ -0,0 +1,55 @@
+set hive.stats.fetch.column.stats=true;
+
+create table if not exists loc_staging (
+  state string,
+  locid int,
+  zip bigint,
+  year int
+) row format delimited fields terminated by '|' stored as textfile;
+
+create table loc_orc like loc_staging;
+alter table loc_orc set fileformat orc;
+
+load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging;
+
+insert overwrite table loc_orc select * from loc_staging;
+
+analyze table loc_orc compute statistics for columns state,locid,zip,year;
+
+-- numRows: 8 rawDataSize: 688
+explain extended select state from loc_orc;
+
+-- numRows: 16 rawDataSize: 1376
+explain extended select * from (select state from loc_orc union all select state from loc_orc) tmp;
+
+-- numRows: 8 rawDataSize: 796
+explain extended select * from loc_orc;
+
+-- numRows: 16 rawDataSize: 1592
+explain extended select * from (select * from loc_orc union all select * from loc_orc) tmp;
+
+create database test;
+use test;
+create table if not exists loc_staging (
+  state string,
+  locid int,
+  zip bigint,
+  year int
+) row format delimited fields terminated by '|' stored as textfile;
+
+create table loc_orc like loc_staging;
+alter table loc_orc set fileformat orc;
+
+load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging;
+
+insert overwrite table loc_orc select * from loc_staging;
+
+analyze table loc_staging compute statistics;
+analyze table loc_staging compute statistics for columns state,locid,zip,year;
+analyze table loc_orc compute statistics for columns state,locid,zip,year;
+
+-- numRows: 16 rawDataSize: 1376
+explain extended select * from (select state from default.loc_orc union all select state from test.loc_orc) temp;
+
+-- numRows: 16 rawDataSize: 1376
+explain extended select * from (select state from test.loc_staging union all select state from test.loc_orc) temp;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ansi_sql_arithmetic.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ansi_sql_arithmetic.q
new file mode 100644
index 0000000000000000000000000000000000000000..3788301ebb3d2740f0de396347900f1b7638c621
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ansi_sql_arithmetic.q
@@ -0,0 +1,13 @@
+
+set hive.compat=latest;
+
+-- With ansi sql arithmetic enabled, int / int => exact numeric type
+explain select cast(key as int) / cast(key as int) from src limit 1;
+select cast(key as int) / cast(key as int) from src limit 1;
+
+
+set hive.compat=0.12;
+
+-- With ansi sql arithmetic disabled, int / int => double
+explain select cast(key as int) / cast(key as int) from src limit 1;
+select cast(key as int) / cast(key as int) from src limit 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q
index b83eab5d38137a9274edb53a881421fae78af886..cc9801d8871b85997c59bf3cedf808b5e5ad1c87 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q
@@ -14,7 +14,7 @@ create table tstsrcpart like srcpart;
 -- to be thrown during the LOAD step. This behavior is now tested in
 -- clientnegative/archive_corrupt.q
 
-load data local inpath '../data/files/archive_corrupt.rc' overwrite into table tstsrcpart partition (ds='2008-04-08', hr='11');
+load data local inpath '../../data/files/archive_corrupt.rc' overwrite into table tstsrcpart partition (ds='2008-04-08', hr='11');
 
 insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='12')
 select key, value from srcpart where ds='2008-04-08' and hr='12';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q
index ddc06a99bb5bc1cbc2a0ef923638999c2d5451ee..50c0faa5e410812422856ebe7952525c4398d885 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q
@@ -1,7 +1,7 @@
 set hive.archive.enabled = true;
 set hive.enforce.bucketing = true;
 
--- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S)
+-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20)
 
 drop table tstsrc;
 drop table tstsrcpart;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_1_sql_std.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_1_sql_std.q
new file mode 100644
index 0000000000000000000000000000000000000000..79ae17ad5da2ccb4946e1aa0c8c1e2acc0cf43b5
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_1_sql_std.q
@@ -0,0 +1,36 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set user.name=hive_admin_user;
+
+create table src_autho_test (key STRING, value STRING) ;
+
+set hive.security.authorization.enabled=true;
+set  role ADMIN; 
+--table grant to user
+
+grant select on table src_autho_test to user user_sauth;
+
+show grant user user_sauth on table src_autho_test;
+
+
+revoke select on table src_autho_test from user user_sauth;
+show grant user user_sauth on table src_autho_test;
+
+--role
+create role src_role;
+grant role src_role to user user_sauth;
+show role grant user user_sauth;
+
+--table grant to role
+
+grant select on table src_autho_test to role src_role;
+
+show grant role src_role on table src_autho_test;
+revoke select on table src_autho_test from role src_role;
+
+-- drop role
+drop role src_role;
+
+set hive.security.authorization.enabled=false;
+drop table src_autho_test;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q
index 4fc79b9ed294a7c680d8c0c7bff44e50fe790337..3353c534e1793888ee46858c048802b5b6535d80 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q
@@ -6,7 +6,7 @@ ALTER TABLE authorization_part SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="T
 set hive.security.authorization.enabled=true;
 
 -- column grant to user
-grant Create on table authorization_part to user hive_test_user;
+grant Create on  authorization_part to user hive_test_user;
 grant Update on table authorization_part to user hive_test_user;
 grant Drop on table authorization_part to user hive_test_user;
 grant select on table src_auth_tmp to user hive_test_user;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_9.q
new file mode 100644
index 0000000000000000000000000000000000000000..1abe659fa447642a20e5e6ca88a779f3ee3581c1
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_9.q
@@ -0,0 +1,17 @@
+-- SORT_BEFORE_DIFF
+
+create table dummy (key string, value string);
+
+grant select on database default to user hive_test_user;
+grant select on table dummy to user hive_test_user;
+grant select (key, value) on table dummy to user hive_test_user;
+
+show grant user hive_test_user on database default;
+show grant user hive_test_user on table dummy;
+show grant user hive_test_user on all;
+
+grant select on database default to user hive_test_user2;
+grant select on table dummy to user hive_test_user2;
+grant select (key, value) on table dummy to user hive_test_user2;
+
+show grant on all;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_admin_almighty1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_admin_almighty1.q
new file mode 100644
index 0000000000000000000000000000000000000000..45c4a7dc85b7279da7cd9f7ce71473ed84c458a8
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_admin_almighty1.q
@@ -0,0 +1,17 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set user.name=hive_test_user;
+
+-- actions from admin should work as if admin has all privileges
+
+create table t1(i int);
+set user.name=hive_admin_user;
+
+show current roles;
+set role ADMIN;
+show current roles;
+select * from t1;
+grant all on table t1 to user user1;
+show grant user user1 on table t1;
+drop table t1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_func1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_func1.q
new file mode 100644
index 0000000000000000000000000000000000000000..65a7b339c2b764aecf86b268d7b64ade1884f41f
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_func1.q
@@ -0,0 +1,14 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=hive_admin_user;
+
+-- admin required for create function
+set role ADMIN;
+
+create temporary function temp_fn as 'org.apache.hadoop.hive.ql.udf.UDFAscii';
+create function perm_fn as 'org.apache.hadoop.hive.ql.udf.UDFAscii';
+
+drop temporary function temp_fn;
+drop function perm_fn;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_macro1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_macro1.q
new file mode 100644
index 0000000000000000000000000000000000000000..fb60500b899ec9ba420fba8aaab7fe3c7ddc95a4
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_macro1.q
@@ -0,0 +1,12 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=hive_admin_user;
+
+-- admin required for create macro
+set role ADMIN;
+
+create temporary macro mymacro1(x double) x * x;
+
+drop temporary macro mymacro1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q
new file mode 100644
index 0000000000000000000000000000000000000000..17f4861cd20f85580481c47fa312e9dc6fd8239d
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q
@@ -0,0 +1,10 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+
+create table create_table_creator_priv_test(i int);
+
+-- all privileges should have been set for user
+
+show grant user user1 on table create_table_creator_priv_test;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_public_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_public_role.q
new file mode 100644
index 0000000000000000000000000000000000000000..8473178cd66076eef7b6c21324a567f07252d515
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_public_role.q
@@ -0,0 +1,18 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE  t_gpr1(i int);
+
+-- all privileges should have been set for user
+
+GRANT ALL ON t_gpr1 TO ROLE public;
+
+SHOW GRANT ON TABLE t_gpr1;
+
+set user.name=user2;
+SHOW CURRENT ROLES;
+-- user2 should be able to do a describe table, as pubic is in the current roles
+DESC t_gpr1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q
new file mode 100644
index 0000000000000000000000000000000000000000..02d364edb488d953646c7774ea02c62feae15ab5
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q
@@ -0,0 +1,43 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE  table_priv1(i int);
+
+-- all privileges should have been set for user
+
+-- grant insert privilege to another user
+GRANT INSERT ON table_priv1 TO USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv1;
+
+-- grant select privilege to another user with grant
+GRANT SELECT ON table_priv1 TO USER user2 with grant option;
+SHOW GRANT USER user2 ON TABLE table_priv1;
+
+set user.name=user2;
+-- change to other user - user2
+-- grant permissions to another user as user2
+GRANT SELECT ON table_priv1 TO USER user3 with grant option;
+SHOW GRANT USER user3 ON TABLE table_priv1;
+
+set user.name=user3;
+-- change to other user - user3
+-- grant permissions to another user as user3
+GRANT SELECT ON table_priv1 TO USER user4 with grant option;
+SHOW GRANT USER user4 ON TABLE table_priv1;
+
+set user.name=user1;
+-- switched back to table owner
+
+-- grant all with grant to user22
+GRANT ALL ON table_priv1 TO USER user22 with grant option;
+SHOW GRANT USER user22 ON TABLE table_priv1;
+
+set user.name=user22;
+
+-- grant all without grant to user33
+GRANT ALL ON table_priv1 TO USER user33 with grant option;
+SHOW GRANT USER user33 ON TABLE table_priv1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_index.q
new file mode 100644
index 0000000000000000000000000000000000000000..1f177ffd1fadbefa06ab00de9162635183a6dc91
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_index.q
@@ -0,0 +1,12 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.stats.dbclass=fs;
+set hive.security.authorization.enabled=true;
+create table t1 (a int);
+create index t1_index on table t1(a) as 'COMPACT' WITH DEFERRED REBUILD;
+desc formatted default__t1_t1_index__;
+alter index t1_index on t1 rebuild;
+
+drop table t1;
+
+set hive.security.authorization.enabled=false;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions.q
new file mode 100644
index 0000000000000000000000000000000000000000..85d8b1114b0104adadcd20030083f8c1d21f16fe
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions.q
@@ -0,0 +1,16 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=user1;
+
+-- actions that require user to be table owner
+create table t1(i int);
+
+ALTER TABLE t1 SET SERDEPROPERTIES ('field.delim' = ',');
+drop table t1;
+
+create table t1(i int);
+create view vt1 as select * from t1;
+
+drop view vt1;
+alter table t1 rename to tnew1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions_db.q
new file mode 100644
index 0000000000000000000000000000000000000000..36ab2600dcd2dc60370ddb22dc1f927fd485040f
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions_db.q
@@ -0,0 +1,21 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=hive_admin_user;
+
+set role admin;
+-- create role, db, make role the owner of db
+create role testrole;
+grant role testrole to user hrt_1;
+create database testdb;
+alter database testdb set owner role testrole;
+desc database testdb;
+
+-- actions that require user to be db owner 
+-- create table
+use testdb;
+create table foobar (foo string, bar string);
+
+-- drop db
+drop database testdb cascade;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_parts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_parts.q
new file mode 100644
index 0000000000000000000000000000000000000000..bee091b1d349cb51f04abfea5a8c517803d32edc
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_parts.q
@@ -0,0 +1,19 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_add_part1;
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_add_part2;
+
+
+
+
+-- check add partition without insert privilege
+create table tpart(i int, j int) partitioned by (k string);
+
+alter table tpart add partition (k = '1') location '${system:test.tmp.dir}/a_uri_add_part1/';
+alter table tpart add partition (k = '2') location '${system:test.tmp.dir}/a_uri_add_part2/';
+
+select count(*) from tpart;
+
+analyze table tpart partition (k) compute statistics;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q
new file mode 100644
index 0000000000000000000000000000000000000000..ccda3b51579796bdfe05a89cda12b0c5575335e1
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q
@@ -0,0 +1,61 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set user.name=user1;
+-- current user has been set (comment line before the set cmd is resulting in parse error!!)
+
+CREATE TABLE table_priv_rev(i int);
+
+-- grant insert privilege to user2
+GRANT INSERT ON table_priv_rev TO USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+SHOW GRANT USER user2 ON ALL;
+
+-- revoke insert privilege from user2
+REVOKE INSERT ON TABLE table_priv_rev FROM USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+-- grant all privileges one at a time --
+-- grant insert privilege to user2
+GRANT INSERT ON table_priv_rev TO USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+SHOW GRANT USER user2 ON ALL;
+
+-- grant select privilege to user2, with grant option
+GRANT SELECT ON table_priv_rev TO USER user2 WITH GRANT OPTION;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+-- grant update privilege to user2
+GRANT UPDATE ON table_priv_rev TO USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+-- grant delete privilege to user2
+GRANT DELETE ON table_priv_rev TO USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+
+-- start revoking --
+-- revoke update privilege from user2
+REVOKE UPDATE ON TABLE table_priv_rev FROM USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+SHOW GRANT USER user2 ON ALL;
+
+-- revoke DELETE privilege from user2
+REVOKE DELETE ON TABLE table_priv_rev FROM USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+-- revoke insert privilege from user2
+REVOKE INSERT ON TABLE table_priv_rev FROM USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+-- revoke select privilege from user2
+REVOKE SELECT ON TABLE table_priv_rev FROM USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+SHOW GRANT USER user2 ON ALL;
+
+-- grant all followed by revoke all
+GRANT ALL ON table_priv_rev TO USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
+
+REVOKE ALL ON TABLE table_priv_rev FROM USER user2;
+SHOW GRANT USER user2 ON TABLE table_priv_rev;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant1.q
new file mode 100644
index 0000000000000000000000000000000000000000..f89d0dc98568894a6cda7eb1fb8cec68c63ed1bd
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant1.q
@@ -0,0 +1,38 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set user.name=hive_admin_user;
+
+-- enable sql standard authorization
+-- role granting without role keyword
+set role ADMIN;
+create role src_role2;
+grant  src_role2 to user user2 ;
+show role grant user user2;
+show roles;
+
+-- revoke role without role keyword
+revoke src_role2 from user user2;
+show role grant user user2;
+show roles;
+
+----------------------------------------
+-- role granting without role keyword, with admin option (syntax check)
+----------------------------------------
+
+create role src_role_wadmin;
+grant  src_role_wadmin to user user2 with admin option;
+show role grant user user2;
+
+-- revoke role without role keyword
+revoke src_role_wadmin from user user2;
+show role grant user user2;
+
+
+
+-- drop roles
+show roles;
+drop role src_role2;
+show roles;
+drop role src_role_wadmin;
+show roles;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant2.q
new file mode 100644
index 0000000000000000000000000000000000000000..984d7ed1d091a90468b5df393226f76e66e629db
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant2.q
@@ -0,0 +1,34 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
+set hive.cli.print.header=true;
+set user.name=hive_admin_user;
+set role ADMIN;
+
+----------------------------------------
+-- role granting with admin option
+----------------------------------------
+
+create role src_role_wadmin;
+grant  src_role_wadmin to user user2 with admin option;
+show role grant user user2;
+show principals src_role_wadmin;
+
+set user.name=user2;
+set role src_role_wadmin;
+grant  src_role_wadmin to user user3;
+show role grant user user3;
+
+set user.name=hive_admin_user;
+set role ADMIN;
+show principals src_role_wadmin;
+
+set user.name=user2;
+set role src_role_wadmin;
+revoke src_role_wadmin from user user3;
+show role grant user user3;
+
+set user.name=hive_admin_user;
+set role ADMIN;
+show principals src_role_wadmin;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q
new file mode 100644
index 0000000000000000000000000000000000000000..6b5af6e94e16aba6b64b461f49714c28ddc52148
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q
@@ -0,0 +1,21 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set user.name=hive_admin_user;
+set role ADMIN;
+show current roles;
+
+create role r1;
+grant role r1 to user hive_admin_user;
+set role r1;
+show current roles;
+
+set role PUBLIC;
+show current roles;
+
+set role ALL;
+show current roles;
+
+set role ADMIN;
+drop role r1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_view_sqlstd.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_view_sqlstd.q
new file mode 100644
index 0000000000000000000000000000000000000000..bd7bbfedf83e8bfdf1adb4563f84846e0311865e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_view_sqlstd.q
@@ -0,0 +1,66 @@
+set hive.users.in.admin.role=hive_admin_user;
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+set hive.security.authorization.enabled=true;
+set user.name=user1;
+
+-- Test view authorization , and 'show grant' variants
+
+create table t1(i int, j int, k int);
+show grant on table t1;
+
+-- protecting certain columns
+create view vt1 as select i,k from t1;
+
+-- protecting certain rows
+create view vt2 as select * from t1 where i > 1;
+
+show grant user user1 on all;
+
+--view grant to user
+-- try with and without table keyword
+
+grant select on vt1 to user user2;
+grant insert on table vt1 to user user3;
+
+show grant user user2 on table vt1;
+show grant user user3 on table vt1;
+
+
+set user.name=user2;
+select * from vt1;
+
+set user.name=user1;
+
+grant all on table vt2 to user user2;
+show grant user user2 on table vt2;
+show grant user user2 on all;
+
+revoke all on vt2 from user user2;
+show grant user user2 on table vt2;
+
+show grant on table vt2;
+
+
+revoke select on table vt1 from user user2;
+show grant user user2 on table vt1;
+
+show grant user user2 on all;
+
+-- grant privileges on roles for view, after next statement
+show grant user user3 on table vt1;
+
+set user.name=hive_admin_user;
+show current roles;
+set role ADMIN;
+create role role_v;
+grant  role_v to user user4 ;
+show role grant user user4;
+show roles;
+
+grant all on table vt2 to role role_v;
+show grant role role_v on table vt2;
+
+revoke delete on table vt2 from role role_v;
+show grant role role_v on table vt2;
+show grant on table vt2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q
index eaf7489a1702544fc5c61ffd8fbc46673b428425..b8734abfd12acc6decfb6596ed8d6e2a378030d9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q
@@ -1,3 +1,5 @@
+set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook;
+
 set hive.auto.convert.join = true;
 set hive.mapjoin.localtask.max.memory.usage = 0.0001;
 set hive.mapjoin.check.memory.rows = 2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q
index 289bfbc6da9176f14a2a8b08e92ce67e42f5a95c..e7846eeecf5c35f3cbb7e583029b86fa688ef827 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q
@@ -32,10 +32,10 @@ from studenttab10k_smb s join votertab10k_smb v
 on (s.name = v.name)
 group by s.name;
 
-load data local inpath '../data/files/empty1.txt' into table studenttab10k_smb;
-load data local inpath '../data/files/empty2.txt' into table studenttab10k_smb;
-load data local inpath '../data/files/empty1.txt' into table votertab10k_smb;
-load data local inpath '../data/files/empty2.txt' into table votertab10k_smb;
+load data local inpath '../../data/files/empty1.txt' into table studenttab10k_smb;
+load data local inpath '../../data/files/empty2.txt' into table studenttab10k_smb;
+load data local inpath '../../data/files/empty1.txt' into table votertab10k_smb;
+load data local inpath '../../data/files/empty2.txt' into table votertab10k_smb;
 
 explain select s.name, count(distinct registration)
 from studenttab10k_smb s join votertab10k_smb v
@@ -51,10 +51,10 @@ group by s.name;
 create table studenttab10k_part (name string, age int, gpa double) partitioned by (p string) clustered by (name) sorted by (name) into 2 buckets;
 create table votertab10k_part (name string, age int, registration string, contributions float) partitioned by (p string) clustered by (name) sorted by (name) into 2 buckets;
 
-load data local inpath '../data/files/empty1.txt' into table studenttab10k_part partition (p='foo');
-load data local inpath '../data/files/empty2.txt' into table studenttab10k_part partition (p='foo');
-load data local inpath '../data/files/empty1.txt' into table votertab10k_part partition (p='foo');
-load data local inpath '../data/files/empty2.txt' into table votertab10k_part partition (p='foo');
+load data local inpath '../../data/files/empty1.txt' into table studenttab10k_part partition (p='foo');
+load data local inpath '../../data/files/empty2.txt' into table studenttab10k_part partition (p='foo');
+load data local inpath '../../data/files/empty1.txt' into table votertab10k_part partition (p='foo');
+load data local inpath '../../data/files/empty2.txt' into table votertab10k_part partition (p='foo');
 
 explain select s.name, count(distinct registration)
 from studenttab10k_part s join votertab10k_part v
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q
index 458504cdc3d040c4aafe453edb2802f25eb7ebea..eefd2111c97b48f9e0466ad9381aa925a0264ce1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q
@@ -1,7 +1,7 @@
 set hive.auto.convert.join = true;
 
 CREATE TABLE myinput1(key int, value int);
-LOAD DATA LOCAL INPATH '../data/files/in3.txt' INTO TABLE myinput1;
+LOAD DATA LOCAL INPATH '../../data/files/in3.txt' INTO TABLE myinput1;
 
 SELECT sum(hash(a.key,a.value,b.key,b.value))  FROM myinput1 a JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value;
 SELECT sum(hash(a.key,a.value,b.key,b.value))  FROM myinput1 a LEFT OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value;
@@ -38,10 +38,10 @@ SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN my
 
 CREATE TABLE smb_input1(key int, value int) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; 
 CREATE TABLE smb_input2(key int, value int) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS; 
-LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input1;
-LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input1;
-LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input2;
-LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input2;
+LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input1;
+LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input1;
+LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input2;
+LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input2;
 
 SET hive.optimize.bucketmapjoin = true;
 SET hive.optimize.bucketmapjoin.sortedmerge = true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q
index 766348d7c001e9eb61e84746dfeeba8eb9aa53f5..d1b7bb40189a48f297cc99f4e02035a1ad6296d3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q
@@ -1,7 +1,7 @@
 set hive.auto.convert.join = true;
 
 CREATE TABLE myinput1(key int, value int);
-LOAD DATA LOCAL INPATH '../data/files/in1.txt' INTO TABLE myinput1;
+LOAD DATA LOCAL INPATH '../../data/files/in1.txt' INTO TABLE myinput1;
 
 SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a JOIN myinput1 b;
 SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN myinput1 b;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q
index 46a4a0d5339a790faa464b980fa182464487cbe8..55bd975803d536fd714e59bcc43ca96cf165e872 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q
@@ -1,13 +1,13 @@
 -- HIVE-5056 RS has expression list for values, but it's ignored in MapJoinProcessor
 
 create table testsrc ( `key` int,`val` string);
-load data local inpath '../data/files/kv1.txt' overwrite into table testsrc;
+load data local inpath '../../data/files/kv1.txt' overwrite into table testsrc;
 drop table if exists orderpayment_small;
 create table orderpayment_small (`dealid` int,`date` string,`time` string, `cityid` int, `userid` int);
-insert overwrite table orderpayment_small select 748, '2011-03-24', '2011-03-24', 55 ,5372613 from testsrc limit 1;
+insert overwrite table orderpayment_small select 748, '2011-03-24', '2011-03-24', 55 ,5372613 from testsrc tablesample (1 rows);
 drop table if exists user_small;
 create table user_small( userid int);
-insert overwrite table user_small select key from testsrc limit 100;
+insert overwrite table user_small select key from testsrc tablesample (100 rows);
 
 set hive.auto.convert.join.noconditionaltask.size = 200;
 explain extended SELECT
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_without_localtask.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_without_localtask.q
new file mode 100644
index 0000000000000000000000000000000000000000..f23e227f1ec90af468411be6cee054873002168a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_without_localtask.q
@@ -0,0 +1,29 @@
+set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook;
+set hive.auto.convert.join=true;
+set hive.auto.convert.join.use.nonstaged=true;
+
+set hive.auto.convert.join.noconditionaltask.size=100;
+
+explain
+select a.* from src a join src b on a.key=b.key limit 40;
+
+select a.* from src a join src b on a.key=b.key limit 40;
+
+explain
+select a.* from src a join src b on a.key=b.key join src c on a.value=c.value limit 40;
+
+select a.* from src a join src b on a.key=b.key join src c on a.value=c.value limit 40;
+
+set hive.auto.convert.join.noconditionaltask.size=100;
+
+explain
+select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40;
+
+select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40;
+
+set hive.mapjoin.localtask.max.memory.usage = 0.0001;
+set hive.mapjoin.check.memory.rows = 2;
+
+-- fallback to common join
+select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q
index e76b560f4189947dbb5faf2a07855a5470db6ada..ddd2c1882ec4d5d2b876dcdda2e54cc8f24cd100 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q
@@ -2,19 +2,19 @@
 
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) 
 CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
 
 set hive.auto.convert.join=true;
 set hive.auto.convert.sortmerge.join=true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q
index f9fa1e4d4e1db6c246df02ef12b098c0605d84fd..da2e26fde7069ffac0a5b341487e7e231a5deed0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q
@@ -1,19 +1,19 @@
 -- small 1 part, 2 bucket & big 2 part, 4 bucket
 
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
 
 set hive.auto.convert.join=true;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q
index db53a656a850aa0b174a130ff7d0f21774db6a2d..f434b33603603f0d0e5ba9fe99146173318c11c8 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q
@@ -2,19 +2,19 @@
 
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string)
 CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
 
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
 
 set hive.auto.convert.join=true;
 set hive.auto.convert.sortmerge.join=true;
@@ -23,9 +23,9 @@ set hive.optimize.bucketmapjoin.sortedmerge = true;
 
 CREATE TABLE bucket_medium (key string, value string) partitioned by (ds string)
 CLUSTERED BY (key) SORTED BY (key) INTO 3 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08');
 
 explain extended select count(*) FROM bucket_small a JOIN bucket_medium b ON a.key = b.key JOIN bucket_big c ON c.key = b.key JOIN bucket_medium d ON c.key = b.key;
 select count(*) FROM bucket_small a JOIN bucket_medium b ON a.key = b.key JOIN bucket_big c ON c.key = b.key JOIN bucket_medium d ON c.key = b.key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q
new file mode 100644
index 0000000000000000000000000000000000000000..7e9555d7917dfbf968af3fedc6387250d90a0a68
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q
@@ -0,0 +1,92 @@
+set hive.auto.convert.join=true;
+
+set hive.exec.dynamic.partition.mode=nonstrict;
+set hive.enforce.bucketing=true;
+set hive.enforce.sorting=true;
+
+set hive.auto.convert.sortmerge.join=true;
+set hive.optimize.bucketmapjoin = true;
+set hive.optimize.bucketmapjoin.sortedmerge = true;
+
+CREATE TABLE stage_bucket_big
+(
+key BIGINT,
+value STRING
+)
+PARTITIONED BY (file_tag STRING);
+
+CREATE TABLE bucket_big
+(
+key BIGINT,
+value STRING
+)
+PARTITIONED BY (day STRING, pri bigint)
+clustered by (key) sorted by (key) into 12 buckets
+stored as RCFile;
+
+CREATE TABLE stage_bucket_small
+(
+key BIGINT,
+value string
+)
+PARTITIONED BY (file_tag STRING);
+
+CREATE TABLE bucket_small
+(
+key BIGINT,
+value string
+)
+PARTITIONED BY (pri bigint)
+clustered by (key) sorted by (key) into 12 buckets
+stored as RCFile;
+
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' overwrite into table stage_bucket_small partition (file_tag='1'); 
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' overwrite into table stage_bucket_small partition (file_tag='2'); 
+
+insert overwrite table bucket_small partition(pri) 
+select 
+key, 
+value, 
+file_tag as pri 
+from 
+stage_bucket_small 
+where file_tag between 1 and 2;
+
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' overwrite into table stage_bucket_big partition (file_tag='1'); 
+
+insert overwrite table bucket_big partition(day,pri) 
+select 
+key, 
+value, 
+'day1' as day, 
+1 as pri 
+from 
+stage_bucket_big 
+where 
+file_tag='1'; 
+
+select 
+a.key , 
+a.value , 
+b.value , 
+'day1' as day, 
+1 as pri 
+from 
+( 
+select 
+key, 
+value 
+from bucket_big where day='day1'
+) a 
+left outer join 
+( 
+select 
+key, 
+value
+from bucket_small 
+where pri between 1 and 2
+) b 
+on 
+(a.key = b.key) 
+; 
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q
index 75339778af802cce9a5f777b09ff7ffc749166d3..eef5483b5347df100b243416c094da0ae27e365e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q
@@ -1,16 +1,16 @@
 -- small 1 part, 4 bucket & big 2 part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
 
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
 
 set hive.auto.convert.join=true;
 set hive.auto.convert.sortmerge.join=true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q
index 39a695fdf039d77fa5fb93873d3056b3460bbe68..c094ecdb6be5ef222a297d3a77094def57965c77 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q
@@ -1,16 +1,16 @@
 -- small 2 part, 2 bucket & big 1 part, 4 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
-load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
 
 set hive.auto.convert.join=true;
 set hive.auto.convert.sortmerge.join=true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q
index 6072272c478a3188ddfabe39c227d5d9055b6471..18acfbfb76543d3579068b16abad7aed798e0baa 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q
@@ -1,18 +1,18 @@
 -- small 2 part, 4 bucket & big 1 part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
-load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
 
 set hive.auto.convert.join=true;
 set hive.auto.convert.sortmerge.join=true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q
index a28ce3de5fc63bb1ddc57beab1db530a2ac36d86..98d6df9b1959110fb18e40d6a030a060202eb8cc 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q
@@ -1,13 +1,13 @@
 -- small no part, 4 bucket & big no part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small;
-load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small;
-load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small;
-load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small;
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small;
+load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small;
+load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small;
+load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small;
 
 CREATE TABLE bucket_big (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big;
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big;
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big;
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big;
 
 set hive.auto.convert.sortmerge.join=true;
 set hive.optimize.bucketmapjoin = true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q
index d62f6377218c60dc55a186e169d02decf1047ddb..e19cc317f36e316b5d9d6dd21eb96966783d6968 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q
@@ -1,21 +1,21 @@
 -- small 2 part, 4 bucket & big 2 part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
-load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
 
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
 
 set hive.auto.convert.join=true;
 set hive.auto.convert.sortmerge.join=true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q
index 6302a1be1477c0c9b2f29d455281e1bc3375b6ff..a66806f21a8a308b81535d4f08746be36cb99b1d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q
@@ -1,21 +1,21 @@
 -- small 2 part, 2 bucket & big 2 part, 4 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
-load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
 
 set hive.auto.convert.join=true;
 set hive.auto.convert.sortmerge.join=true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q
index 83672062319746df2b359fcaca8af71b7b4e3564..cb6f173ccfa76420f07c7d7e69749482537127aa 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q
@@ -35,7 +35,7 @@ TBLPROPERTIES ('avro.schema.literal'='{
   ]
 }');
 
-LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors4;
+LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors4;
 
 set hive.exec.compress.output=true;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q
index 3fe8ff12b4506e8c9563e084380399fcabc2edb8..f723cbcc608fb3e54cf040436c4f5a55a696cc55 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q
@@ -36,7 +36,7 @@ TBLPROPERTIES ('avro.schema.literal'='{
 
 DESCRIBE doctors_with_new_field;
 
-LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors_with_new_field;
+LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors_with_new_field;
 
 SELECT * FROM doctors_with_new_field ORDER BY first_name;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q
index 25b77c06270d0329632917b329e3fb03e6f6c100..4c33a834668b07867756396446093787c629f2cc 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q
@@ -37,7 +37,7 @@ TBLPROPERTIES ('avro.schema.literal'='{
 
 DESCRIBE doctors4;
 
-LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors4;
+LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors4;
 
 CREATE TABLE episodes
 ROW FORMAT
@@ -70,7 +70,7 @@ TBLPROPERTIES ('avro.schema.literal'='{
 
 DESCRIBE episodes;
 
-LOAD DATA LOCAL INPATH '../data/files/episodes.avro' INTO TABLE episodes;
+LOAD DATA LOCAL INPATH '../../data/files/episodes.avro' INTO TABLE episodes;
 
 SELECT e.title, e.air_date, d.first_name, d.last_name, d.extra_field, e.air_date
 FROM doctors4 d JOIN episodes e ON (d.number=e.doctor)
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q
index 584c6f740ba91722462455fe71beb080b990bce4..f90ceb96f5db93495398f9e736dd3049e7bfeac4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q
@@ -17,7 +17,7 @@ CREATE TABLE test_serializer(string1 STRING,
  ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' COLLECTION ITEMS TERMINATED BY ':' MAP KEYS TERMINATED BY '#' LINES TERMINATED BY '\n'
  STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/csv.txt' INTO TABLE test_serializer;
+LOAD DATA LOCAL INPATH '../../data/files/csv.txt' INTO TABLE test_serializer;
 
 CREATE TABLE as_avro
   ROW FORMAT
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q
index 8e4d40f2bd59bf57b01f60674dd4451626230188..6fe5117026ce8693cc50dc1e682d998549198787 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q
@@ -28,7 +28,7 @@ TBLPROPERTIES ('avro.schema.literal'='{
   ]
 }');
 
-LOAD DATA LOCAL INPATH '../data/files/episodes.avro' INTO TABLE episodes;
+LOAD DATA LOCAL INPATH '../../data/files/episodes.avro' INTO TABLE episodes;
 
 CREATE TABLE episodes_partitioned
 PARTITIONED BY (doctor_pt INT)
@@ -66,7 +66,7 @@ INSERT OVERWRITE TABLE episodes_partitioned PARTITION (doctor_pt) SELECT title,
 SELECT * FROM episodes_partitioned WHERE doctor_pt > 6 ORDER BY air_date;
 
 -- Verify that Fetch works in addition to Map
-SELECT * FROM episodes_partitioned LIMIT 5;
+SELECT * FROM episodes_partitioned ORDER BY air_date LIMIT 5;
 -- Fetch w/filter to specific partition
 SELECT * FROM episodes_partitioned WHERE doctor_pt = 6;
 -- Fetch w/non-existant partition
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q
index e3f8b07b30c597e1044277e70c72995ccc59bdcc..dbb999503b60f2ef1152adac11fe0a2ce0f87e6e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q
@@ -30,7 +30,7 @@ TBLPROPERTIES ('avro.schema.literal'='{
 
 DESCRIBE doctors;
 
-LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors;
+LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors;
 
 SELECT * FROM doctors ORDER BY number;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q
index e0a8b95401d0b313bdfd7c3af96b8ad00e40f906..4f80dc33c96665a48ea3fd8cd3b108a19492ae95 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q
@@ -1 +1,3 @@
-select cast(cast('a' as binary) as string) from src limit 1;
+set hive.fetch.task.conversion=more;
+
+select cast(cast('a' as binary) as string) from src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q
index eadf07d14d7567ce88056a23bea5ae22d28a2baf..1f5c98a23918f267438499340a0196150a80caaf 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q
@@ -3,6 +3,7 @@ drop table ba_test;
 -- Everything in ba_table1.q + columnar serde in RCFILE.
 
 create table ba_test (ba_key binary, ba_val binary) stored as rcfile;
+alter table ba_test set serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe';
 
 describe extended ba_test;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q
index a98a2305cfcaf1ce6b211966f97890c32ca53156..39c1d25e73397811d95bbb27d5b72cc699916197 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q
@@ -3,7 +3,7 @@ ROW FORMAT DELIMITED
 FIELDS TERMINATED BY '9'
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/string.txt' INTO TABLE mytable;
+LOAD DATA LOCAL INPATH '../../data/files/string.txt' INTO TABLE mytable;
 
 EXPLAIN
 SELECT REGEXP_REPLACE(REGEXP_REPLACE(REGEXP_REPLACE(key, '\001', '^A'), '\0', '^@'), '\002', '^B'), value
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_if_with_path_filter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_if_with_path_filter.q
new file mode 100644
index 0000000000000000000000000000000000000000..956a61f7bd3c1bf934a53a8bc162348ab8b89a99
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_if_with_path_filter.q
@@ -0,0 +1,15 @@
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/bmjpathfilter;
+
+create table t1 (dt string) location '${system:test.tmp.dir}/bmjpathfilter/t1';
+Create table t2 (dt string) stored as orc; 
+dfs -touchz ${system:test.tmp.dir}/bmjpathfilter/t1/_SUCCESS;
+
+SET hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; 
+SET hive.optimize.bucketmapjoin=true; 
+
+SELECT /*+ MAPJOIN(b) */ a.dt FROM t1 a JOIN t2 b ON (a.dt = b.dt);
+ 
+SET hive.optimize.bucketmapjoin=false;
+set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+
+dfs -rmr ${system:test.tmp.dir}/bmjpathfilter;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q
index 33dd5d5cd2e0a8c9e565f1b29452923221455ae9..6bdb09ed64b073af659d3d915ceadb721f6178bd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q
@@ -9,8 +9,8 @@ sorted by (key, value) into 1 BUCKETS stored as textfile;
 create table table2(key string, value string) clustered by (value, key)
 sorted by (value, key) into 1 BUCKETS stored as textfile;
 
-load data local inpath '../data/files/SortCol1Col2.txt' overwrite into table table1;
-load data local inpath '../data/files/SortCol2Col1.txt' overwrite into table table2;
+load data local inpath '../../data/files/SortCol1Col2.txt' overwrite into table table1;
+load data local inpath '../../data/files/SortCol2Col1.txt' overwrite into table table2;
 
 set hive.optimize.bucketmapjoin = true;
 set hive.optimize.bucketmapjoin.sortedmerge = true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q
index d1097e70a9880c3dced5866deaf3fae7bc3d800a..07f6d150ea053520347de804a2c2420c7da92007 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q
@@ -9,8 +9,8 @@ sorted by (key desc, value desc) into 1 BUCKETS stored as textfile;
 create table table2(key string, value string) clustered by (value, key)
 sorted by (value desc, key desc) into 1 BUCKETS stored as textfile;
 
-load data local inpath '../data/files/SortCol1Col2.txt' overwrite into table table1;
-load data local inpath '../data/files/SortCol2Col1.txt' overwrite into table table2;
+load data local inpath '../../data/files/SortCol1Col2.txt' overwrite into table table1;
+load data local inpath '../../data/files/SortCol2Col1.txt' overwrite into table table2;
 
 set hive.optimize.bucketmapjoin = true;
 set hive.optimize.bucketmapjoin.sortedmerge = true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
new file mode 100644
index 0000000000000000000000000000000000000000..c9266a59c342a33a7edea9584ab8c69bcf020b41
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q
@@ -0,0 +1,85 @@
+set hive.auto.convert.join=true;
+set hive.auto.convert.join.noconditionaltask=true;
+set hive.auto.convert.join.noconditionaltask.size=10000;
+
+CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
+CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
+CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
+
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08');
+
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+
+set hive.enforce.bucketing=true;
+set hive.enforce.sorting = true;
+set hive.optimize.bucketingsorting=false;
+insert overwrite table tab_part partition (ds='2008-04-08')
+select key,value from srcbucket_mapjoin_part;
+
+CREATE TABLE tab(key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
+insert overwrite table tab partition (ds='2008-04-08')
+select key,value from srcbucket_mapjoin;
+
+set hive.convert.join.bucket.mapjoin.tez = true;
+explain
+select a.key, a.value, b.value
+from tab a join tab_part b on a.key = b.key;
+
+-- one side is really bucketed. srcbucket_mapjoin is not really a bucketed table.
+-- In this case the sub-query is chosen as the big table.
+explain
+select a.k1, a.v1, b.value
+from (select sum(substr(srcbucket_mapjoin.value,5)) as v1, key as k1 from srcbucket_mapjoin GROUP BY srcbucket_mapjoin.key) a
+join tab b on a.k1 = b.key;
+
+explain
+select a.k1, a.v1, b.value
+from (select sum(substr(tab.value,5)) as v1, key as k1 from tab_part join tab on tab_part.key = tab.key GROUP BY tab.key) a
+join tab b on a.k1 = b.key;
+
+explain
+select a.k1, a.v1, b.value
+from (select sum(substr(x.value,5)) as v1, x.key as k1 from tab x join tab y on x.key = y.key GROUP BY x.key) a
+join tab_part b on a.k1 = b.key;
+
+-- multi-way join
+explain
+select a.key, a.value, b.value
+from tab_part a join tab b on a.key = b.key join tab c on a.key = c.key;
+
+explain
+select a.key, a.value, c.value
+from (select x.key, x.value from tab_part x join tab y on x.key = y.key) a join tab c on a.key = c.key;
+
+-- in this case sub-query is the small table
+explain
+select a.key, a.value, b.value
+from (select key, sum(substr(srcbucket_mapjoin.value,5)) as value from srcbucket_mapjoin GROUP BY srcbucket_mapjoin.key) a
+join tab_part b on a.key = b.key;
+
+set hive.map.aggr=false;
+explain
+select a.key, a.value, b.value
+from (select key, sum(substr(srcbucket_mapjoin.value,5)) as value from srcbucket_mapjoin GROUP BY srcbucket_mapjoin.key) a
+join tab_part b on a.key = b.key;
+
+-- join on non-bucketed column results in broadcast join.
+explain
+select a.key, a.value, b.value
+from tab a join tab_part b on a.value = b.value;
+
+CREATE TABLE tab1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
+insert overwrite table tab1
+select key,value from srcbucket_mapjoin;
+
+explain
+select a.key, a.value, b.value
+from tab1 a join tab_part b on a.key = b.key;
+
+explain select a.key, b.key from tab_part a join tab_part c on a.key = c.key join tab_part b on a.value = b.value;
+
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q
new file mode 100644
index 0000000000000000000000000000000000000000..a3588ec94ccee7ff32bbeb749c2b2b10e6d8610f
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q
@@ -0,0 +1,50 @@
+set hive.auto.convert.join=true;
+set hive.auto.convert.join.noconditionaltask=true;
+set hive.auto.convert.join.noconditionaltask.size=10000;
+
+CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
+CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
+CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
+
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08');
+
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+
+set hive.enforce.bucketing=true;
+set hive.enforce.sorting = true;
+set hive.optimize.bucketingsorting=false;
+insert overwrite table tab_part partition (ds='2008-04-08')
+select key,value from srcbucket_mapjoin_part;
+
+CREATE TABLE tab(key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
+insert overwrite table tab partition (ds='2008-04-08')
+select key,value from srcbucket_mapjoin;
+
+set hive.convert.join.bucket.mapjoin.tez = true;
+
+explain select a.key, b.key from tab_part a join tab_part c on a.key = c.key join tab_part b on a.value = b.value;
+
+CREATE TABLE tab1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
+insert overwrite table tab1
+select key,value from srcbucket_mapjoin;
+
+explain
+select a.key, a.value, b.value
+from tab1 a join src b on a.key = b.key;
+
+explain
+select a.key, b.key from (select key from tab_part where key > 1) a join (select key from tab_part where key > 2) b on a.key = b.key;
+
+explain
+select a.key, b.key from (select key from tab_part where key > 1) a left outer join (select key from tab_part where key > 2) b on a.key = b.key;
+
+explain
+select a.key, b.key from (select key from tab_part where key > 1) a right outer join (select key from tab_part where key > 2) b on a.key = b.key;
+
+explain select a.key, b.key from (select distinct key from tab) a join tab b on b.key = a.key;
+
+explain select a.value, b.value from (select distinct value from tab) a join tab b on b.key = a.value;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q
index 5b013995d5ba4542ab3f64db734ca9050ede3f33..047a2a5230ae0fc7aebbd9eb482b936250b99914 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q
@@ -1,18 +1,18 @@
 -- small 1 part, 2 bucket & big 2 part, 4 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
 
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
 
 set hive.optimize.bucketmapjoin = true;
 explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q
index f952f2ee6d445b7740c021b46880fedebb546e9c..d58e8448a09a84f15f6a0b20c6b9c93108118a1b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q
@@ -1,16 +1,16 @@
 -- small 1 part, 4 bucket & big 2 part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
 
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
 
 set hive.optimize.bucketmapjoin = true;
 explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q
index 461fbb196e758e8c20805c37f67706756cd87f53..fd80174f23119ed8e9506eff0b0b3b671bed9bb6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q
@@ -1,16 +1,16 @@
 -- small 2 part, 2 bucket & big 1 part, 4 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
 
 set hive.optimize.bucketmapjoin = true;
 explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q
index 366da4473b36d90e8e7517e61f1607c50a366b94..5d21ea5d63218674f4f2cee234cfe28da9f2945f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q
@@ -1,18 +1,18 @@
 -- small 2 part, 4 bucket & big 1 part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
 
 set hive.optimize.bucketmapjoin = true;
 explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q
index 411fed392a45e1198abcc0208873bfc883e53312..5078072677be6135c31370364d72fb10df946115 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q
@@ -1,13 +1,13 @@
 -- small no part, 4 bucket & big no part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small;
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small;
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small;
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small;
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small;
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small;
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small;
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small;
 
 CREATE TABLE bucket_big (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big;
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big;
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big;
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big;
 
 set hive.optimize.bucketmapjoin = true;
 explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q
index 204d1e9010d5d97ce543ce296c327076a711c68c..0f7c72f4eaf43832c77059382f2f7a360d505b56 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q
@@ -1,16 +1,16 @@
 -- small no part, 4 bucket & big 2 part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small;
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small;
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small;
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small;
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small;
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small;
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small;
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small;
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
 
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
 
 set hive.optimize.bucketmapjoin = true;
 explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q
index b0bca460f303fa3c540835344aab7a3df76ead4e..c528f61302743d7bb796c233eab725ca24110dd3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q
@@ -1,21 +1,21 @@
 -- small 2 part, 4 bucket & big 2 part, 2 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
 
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
 
 set hive.optimize.bucketmapjoin = true;
 explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q
index 9533c5512f3dd1e3fc31ec5c855f63509ac0b1e1..27c55a36f445b74ec7a76831e8dd9c3eed3ca968 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q
@@ -1,21 +1,21 @@
 -- small 2 part, 2 bucket & big 2 part, 4 bucket
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
 
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
 
 set hive.optimize.bucketmapjoin = true;
 explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q
index 4c7f0c9595488ec2dd72f3da942d86a4a1e4e784..d2e12e82d4a264927ffcf26da00cd1750ae3bdc5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q
@@ -3,7 +3,7 @@ set mapred.min.split.size = 64;
 
 CREATE TABLE T1(name STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(name STRING) STORED AS SEQUENCEFILE;
 
@@ -26,8 +26,8 @@ EXPLAIN SELECT COUNT(1) FROM T2;
 SELECT COUNT(1) FROM T2;
 
 CREATE TABLE T3(name STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T3;
-LOAD DATA LOCAL INPATH '../data/files/kv2.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/kv2.txt' INTO TABLE T3;
 
 EXPLAIN SELECT COUNT(1) FROM T3;
 SELECT COUNT(1) FROM T3;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q
index 9cdfe8e6e9f6e49abb8b536e24d1d9bea5b83282..8b7535dce1ee26b44cba548dc3a9585fe024c3d3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q
@@ -1,17 +1,17 @@
 CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08');
 
 CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08');
 
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09');
 
 set hive.optimize.bucketmapjoin = true;
 select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q
index 2bd8e1e2e5d987457d85a1a5b5f44fc74431077e..204e75988fb544155d1eabd8a6108cd9f67b322f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q
@@ -25,16 +25,16 @@ select /*+mapjoin(a)*/ a.key, a.value, b.value
 from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b
 on a.key=b.key where b.ds="2008-04-08";
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
 
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
 
 create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint);
 create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q
index cf4222bdd781ed75a7567e714a87993ff7f78e7f..09c0ae2bfcbc19a5cf6ca59fa13aff73629d8e48 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q
@@ -2,23 +2,23 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 
 CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) 
 CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
 
 ALTER TABLE srcbucket_mapjoin_part_1 CLUSTERED BY (key) INTO 3 BUCKETS;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2');
 
 CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) 
 CLUSTERED BY (key) INTO 3 BUCKETS STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
 
 ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 2 BUCKETS;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2');
 
 ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 3 BUCKETS;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q
index e10ab522a296d57fa2abfb7af0310e913ec56a23..d330b7718c5f280bc7c957277fa9e587541b86a7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q
@@ -2,25 +2,25 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 
 CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) 
 CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
 
 ALTER TABLE srcbucket_mapjoin_part_1 CLUSTERED BY (key) INTO 4 BUCKETS;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2');
 
 CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) 
 CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
 
 ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 2 BUCKETS;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2');
 
 
 set hive.optimize.bucketmapjoin=true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q
index 8139000f4a733d53cd6e156c80c0153fc54da6ea..43a9de4e8edeff3a951c758869b19de7e36a749e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q
@@ -2,20 +2,20 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 
 CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) 
 CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
 
 CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) 
 CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
 
 ALTER TABLE srcbucket_mapjoin_part_2 NOT CLUSTERED;
 
 CREATE TABLE srcbucket_mapjoin_part_3 (key INT, value STRING) PARTITIONED BY (part STRING)
 STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_3 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_3 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_3 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_3 PARTITION (part='1');
 
 ALTER TABLE srcbucket_mapjoin_part_3 CLUSTERED BY (key) INTO 2 BUCKETS;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q
index fdbadfcb04c9c2a2ec3d26262583664034ac5025..108b67a9f7644746ef7de3321ee95d82be2f433b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q
@@ -1,12 +1,12 @@
 CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
 
 CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
 
 create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint);
 create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint);
@@ -76,8 +76,8 @@ from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b
 on a.key = b.key;
 
 -- HIVE-3210
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09');
 
 set hive.optimize.bucketmapjoin = true;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q
index 8fda80231522e259d444accd1e484530cbaeba0f..78c23d5132503dc9ec4548c575dcdb4aeecb19cb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q
@@ -1,16 +1,16 @@
 CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;
 
 CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
 
 CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
 
 create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint);
 create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q
index c1a8f2ab7bfa064614a04979931d3f2ef1fe83c9..54626e7c48ded399d597abf22a3fb683a9c529bb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q
@@ -1,17 +1,17 @@
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;
 
 CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
 
 CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
 
 create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint);
 create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q
index 2df49b64f3227dbafdf5e0276e3cc86bf334ad91..72cffc2da28743b9be50b37dd1ff14001da33d5f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q
@@ -1,22 +1,22 @@
 CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;
 
 CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09');
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09');
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09');
 
 CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09');
 
 create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint);
 create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q
index 3a96c642b3e4c6f0e733a463e7a6064351620e6c..a15570b77687f49a8b7bb67e5ca795ae3c322acd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q
@@ -2,13 +2,13 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 
 CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) 
 CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0');
 
 CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) 
 CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0');
 
 set hive.optimize.bucketmapjoin=true;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q
index 5e8daa5f8f125afb9507c44bd260859ab94284de..f467ea6cb3ea17fa62b861677e738f5b9f8394fc 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q
@@ -2,13 +2,13 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 
 CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) 
 CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
 
 CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) 
 CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
 
 ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 3 BUCKETS;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q
index 86344a53f5a95b6ccb5d8c0c5ec574f3cfcff4a1..f1d5f581848d07576e3d7184b71cb8794d999c41 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q
@@ -2,14 +2,14 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 
 CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) 
 CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1');
 
 CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) 
 CLUSTERED BY (key) INTO 3 BUCKETS STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
 
 ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 2 BUCKETS;
 
@@ -28,8 +28,8 @@ ON a.key = b.key AND a.part = '1' and b.part = '1';
 
 ALTER TABLE srcbucket_mapjoin_part_2 DROP PARTITION (part='1');
 ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (value) INTO 2 BUCKETS;
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1');
 
 ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 2 BUCKETS;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q
index d7634333e2fee5f3037898be1c91fe3c80980653..ea140ddda91fe63777daea39367cbf0215e0be95 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q
@@ -3,13 +3,13 @@
 
 
 CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;
 
 CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 3 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
 
 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q
index 901f0567590ec3f5a5b46660d27a9d1ed76d4bd3..e2c0d8c5917197492d1cd118e20a6619cdd43edf 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q
@@ -1,12 +1,12 @@
 CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;
 
 CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09');
 
 set hive.optimize.bucketmapjoin = true;
 create table bucketmapjoin_tmp_result (key string , value1 string, value2 string);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q
index d66e1238bb79b94dc1e27f290e72e0699de0536c..6398fff909eab45d4b004c8703761a00ed247993 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q
@@ -8,21 +8,21 @@ create table test2 (key string, value string) clustered by (value) sorted by (va
 create table test3 (key string, value string) clustered by (key, value) sorted by (key, value) into 3 buckets;
 create table test4 (key string, value string) clustered by (value, key) sorted by (value, key) into 3 buckets;
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE test1;
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE test1;
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE test1;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE test1;
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE test1;
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE test1;
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE test2;
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE test2;
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE test2;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE test2;
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE test2;
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE test2;
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE test3;
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE test3;
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE test3;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE test3;
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE test3;
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE test3;
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE test4;
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE test4;
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE test4;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE test4;
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE test4;
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE test4;
 
 set hive.optimize.bucketmapjoin = true;
 -- should be allowed
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q
index 729ffdc8687b51f119fe60059faa91d1b7f55496..b1551f2feb900533944beafecb247e0785a4fbfa 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 -- cast string floats to integer types
 select
   cast('1' as float),
@@ -27,4 +29,4 @@ select
   cast('127' as tinyint),
   cast('1.0a' as int),
   cast('-1.-1' as int)
-from src limit 1;
+from src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_1.q
new file mode 100644
index 0000000000000000000000000000000000000000..840a84a7b3e3045fceef0f08cd0003f53ecb0bad
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_1.q
@@ -0,0 +1,32 @@
+drop table char1;
+drop table char1_1;
+
+create table char1 (key char(10), value char(20));
+create table char1_1 (key string, value string);
+
+-- load from file
+load data local inpath '../../data/files/srcbucket0.txt' overwrite into table char1;
+select * from char1 order by key, value limit 2;
+
+-- insert overwrite, from same/different length char
+insert overwrite table char1
+  select cast(key as char(10)), cast(value as char(15)) from src order by key, value limit 2;
+select key, value from char1 order by key, value;
+
+-- insert overwrite, from string
+insert overwrite table char1
+  select key, value from src order by key, value limit 2;
+select key, value from char1 order by key, value;
+
+-- insert string from char
+insert overwrite table char1_1
+  select key, value from char1 order by key, value limit 2;
+select key, value from char1_1 order by key, value;
+
+-- respect string length
+insert overwrite table char1 
+  select key, cast(value as char(3)) from src order by key, value limit 2;
+select key, value from char1 order by key, value;
+
+drop table char1;
+drop table char1_1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_2.q
new file mode 100644
index 0000000000000000000000000000000000000000..3e4900cb7c949b5c78788cb28ec19c39d5df2e7e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_2.q
@@ -0,0 +1,36 @@
+drop table char_2;
+
+create table char_2 (
+  key char(10),
+  value char(20)
+);
+
+insert overwrite table char_2 select * from src;
+
+select value, sum(cast(key as int)), count(*) numrows
+from src
+group by value
+order by value asc
+limit 5;
+
+-- should match the query from src
+select value, sum(cast(key as int)), count(*) numrows
+from char_2
+group by value
+order by value asc
+limit 5;
+
+select value, sum(cast(key as int)), count(*) numrows
+from src
+group by value
+order by value desc
+limit 5;
+
+-- should match the query from src
+select value, sum(cast(key as int)), count(*) numrows
+from char_2
+group by value
+order by value desc
+limit 5;
+
+drop table char_2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_cast.q
new file mode 100644
index 0000000000000000000000000000000000000000..7f44d4d508537a1394a9fcbab8fbd7bae534a80e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_cast.q
@@ -0,0 +1,92 @@
+
+-- Cast from char to other data types
+select
+  cast(cast('11' as string) as tinyint),
+  cast(cast('11' as string) as smallint),
+  cast(cast('11' as string) as int),
+  cast(cast('11' as string) as bigint),
+  cast(cast('11.00' as string) as float),
+  cast(cast('11.00' as string) as double),
+  cast(cast('11.00' as string) as decimal)
+from src limit 1;
+
+select
+  cast(cast('11' as char(10)) as tinyint),
+  cast(cast('11' as char(10)) as smallint),
+  cast(cast('11' as char(10)) as int),
+  cast(cast('11' as char(10)) as bigint),
+  cast(cast('11.00' as char(10)) as float),
+  cast(cast('11.00' as char(10)) as double),
+  cast(cast('11.00' as char(10)) as decimal)
+from src limit 1;
+
+select
+  cast(cast('2011-01-01' as string) as date),
+  cast(cast('2011-01-01 01:02:03' as string) as timestamp)
+from src limit 1;
+
+select
+  cast(cast('2011-01-01' as char(10)) as date),
+  cast(cast('2011-01-01 01:02:03' as char(30)) as timestamp)
+from src limit 1;
+
+-- no tests from string/char to boolean, that conversion doesn't look useful
+select
+  cast(cast('abc123' as string) as string),
+  cast(cast('abc123' as string) as varchar(10)),
+  cast(cast('abc123' as string) as char(10))
+from src limit 1;
+
+select
+  cast(cast('abc123' as char(10)) as string),
+  cast(cast('abc123' as char(10)) as varchar(10)),
+  cast(cast('abc123' as char(10)) as char(10))
+from src limit 1;
+
+select
+  cast(cast('abc123' as varchar(10)) as string),
+  cast(cast('abc123' as varchar(10)) as varchar(10)),
+  cast(cast('abc123' as varchar(10)) as char(10))
+from src limit 1;
+
+-- cast from other types to char
+select
+  cast(cast(11 as tinyint) as string),
+  cast(cast(11 as smallint) as string),
+  cast(cast(11 as int) as string),
+  cast(cast(11 as bigint) as string),
+  cast(cast(11.00 as float) as string),
+  cast(cast(11.00 as double) as string),
+  cast(cast(11.00 as decimal) as string)
+from src limit 1;
+
+select
+  cast(cast(11 as tinyint) as char(10)),
+  cast(cast(11 as smallint) as char(10)),
+  cast(cast(11 as int) as char(10)),
+  cast(cast(11 as bigint) as char(10)),
+  cast(cast(11.00 as float) as char(10)),
+  cast(cast(11.00 as double) as char(10)),
+  cast(cast(11.00 as decimal) as char(10))
+from src limit 1;
+
+select
+  cast(date '2011-01-01' as string),
+  cast(timestamp('2011-01-01 01:02:03') as string)
+from src limit 1;
+
+select
+  cast(date '2011-01-01' as char(10)),
+  cast(timestamp('2011-01-01 01:02:03') as char(30))
+from src limit 1;
+
+select
+  cast(true as string),
+  cast(false as string)
+from src limit 1;
+
+select
+  cast(true as char(10)),
+  cast(false as char(10))
+from src limit 1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_comparison.q
new file mode 100644
index 0000000000000000000000000000000000000000..e1cfdb2bdb07ae631d8860b276b04b3700027a0e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_comparison.q
@@ -0,0 +1,40 @@
+
+-- Should all be true
+select
+  cast('abc' as char(10)) =  cast('abc' as char(10)),
+  cast('abc' as char(10)) <= cast('abc' as char(10)),
+  cast('abc' as char(10)) >= cast('abc' as char(10)),
+  cast('abc' as char(10)) <  cast('abd' as char(10)),
+  cast('abc' as char(10)) >  cast('abb' as char(10)),
+  cast('abc' as char(10)) <> cast('abb' as char(10))
+from src limit 1;
+
+-- Different char lengths should still compare the same
+select
+  cast('abc' as char(10)) =  cast('abc' as char(3)),
+  cast('abc' as char(10)) <= cast('abc' as char(3)),
+  cast('abc' as char(10)) >= cast('abc' as char(3)),
+  cast('abc' as char(10)) <  cast('abd' as char(3)),
+  cast('abc' as char(10)) >  cast('abb' as char(3)),
+  cast('abc' as char(10)) <> cast('abb' as char(3))
+from src limit 1;
+
+-- Should work with string types as well
+select
+  cast('abc' as char(10)) =  'abc',
+  cast('abc' as char(10)) <= 'abc',
+  cast('abc' as char(10)) >= 'abc',
+  cast('abc' as char(10)) <  'abd',
+  cast('abc' as char(10)) >  'abb',
+  cast('abc' as char(10)) <> 'abb'
+from src limit 1;
+
+-- leading space is significant for char
+select
+  cast(' abc' as char(10)) <> cast('abc' as char(10))
+from src limit 1;
+
+-- trailing space is not significant for char
+select
+  cast('abc ' as char(10)) = cast('abc' as char(10))
+from src limit 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_join1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_join1.q
new file mode 100644
index 0000000000000000000000000000000000000000..373352dee00780a1f41ccc0e353fa902616f9c98
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_join1.q
@@ -0,0 +1,35 @@
+drop table char_join1_ch1;
+drop table char_join1_ch2;
+drop table char_join1_str;
+
+create table  char_join1_ch1 (
+  c1 int,
+  c2 char(10)
+);
+
+create table  char_join1_ch2 (
+  c1 int,
+  c2 char(20)
+);
+
+create table  char_join1_str (
+  c1 int,
+  c2 string
+);
+
+load data local inpath '../../data/files/vc1.txt' into table char_join1_ch1;
+load data local inpath '../../data/files/vc1.txt' into table char_join1_ch2;
+load data local inpath '../../data/files/vc1.txt' into table char_join1_str;
+
+-- Join char with same length char
+select * from char_join1_ch1 a join char_join1_ch1 b on (a.c2 = b.c2) order by a.c1;
+
+-- Join char with different length char
+select * from char_join1_ch1 a join char_join1_ch2 b on (a.c2 = b.c2) order by a.c1;
+
+-- Join char with string
+select * from char_join1_ch1 a join char_join1_str b on (a.c2 = b.c2) order by a.c1;
+
+drop table char_join1_ch1;
+drop table char_join1_ch2;
+drop table char_join1_str;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_nested_types.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_nested_types.q
new file mode 100644
index 0000000000000000000000000000000000000000..c710b6c73174bfaa418b5fbe11fc641e10940c30
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_nested_types.q
@@ -0,0 +1,53 @@
+drop table char_nested_1;
+drop table char_nested_array;
+drop table char_nested_map;
+drop table char_nested_struct;
+drop table char_nested_cta;
+drop table char_nested_view;
+
+create table char_nested_1 (key int, value char(20));
+insert overwrite table char_nested_1
+  select key, value from src order by key limit 1;
+
+-- arrays
+create table char_nested_array (c1 array<char(20)>);
+insert overwrite table char_nested_array
+  select array(value, value) from char_nested_1;
+describe char_nested_array;
+select * from char_nested_array;
+
+-- maps
+create table char_nested_map (c1 map<int, char(20)>);
+insert overwrite table char_nested_map
+  select map(key, value) from char_nested_1;
+describe char_nested_map;
+select * from char_nested_map;
+
+-- structs
+create table char_nested_struct (c1 struct<a:int, b:char(20), c:string>);
+insert overwrite table char_nested_struct
+  select named_struct('a', key,
+                      'b', value,
+                      'c', cast(value as string))
+  from char_nested_1;
+describe char_nested_struct;
+select * from char_nested_struct;
+
+-- nested type with create table as
+create table char_nested_cta as 
+  select * from char_nested_struct;
+describe char_nested_cta;
+select * from char_nested_cta;
+
+-- nested type with view
+create table char_nested_view as 
+  select * from char_nested_struct;
+describe char_nested_view;
+select * from char_nested_view;
+
+drop table char_nested_1;
+drop table char_nested_array;
+drop table char_nested_map;
+drop table char_nested_struct;
+drop table char_nested_cta;
+drop table char_nested_view;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_serde.q
new file mode 100644
index 0000000000000000000000000000000000000000..4340b4de6d597bbf15d5884de8ecb9da7541503c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_serde.q
@@ -0,0 +1,102 @@
+drop table if exists char_serde_regex;
+drop table if exists char_serde_lb;
+drop table if exists char_serde_ls;
+drop table if exists char_serde_c;
+drop table if exists char_serde_lbc;
+drop table if exists char_serde_orc;
+
+--
+-- RegexSerDe
+--
+create table  char_serde_regex (
+  key char(15),
+  value char(20)
+)
+row format serde 'org.apache.hadoop.hive.serde2.RegexSerDe'
+with serdeproperties (
+  "input.regex" = "([^]*)([^]*)"
+)
+stored as textfile;
+
+load data local inpath '../../data/files/srcbucket0.txt' overwrite into table char_serde_regex;
+
+select * from char_serde_regex limit 5;
+select value, count(*) from char_serde_regex group by value limit 5;
+
+--
+-- LazyBinary
+--
+create table  char_serde_lb (
+  key char(15),
+  value char(20)
+);
+alter table char_serde_lb set serde 'org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe';
+
+insert overwrite table char_serde_lb
+  select key, value from char_serde_regex;
+select * from char_serde_lb limit 5;
+select value, count(*) from char_serde_lb group by value limit 5;
+
+--
+-- LazySimple
+--
+create table  char_serde_ls (
+  key char(15),
+  value char(20)
+);
+alter table char_serde_ls set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe';
+
+insert overwrite table char_serde_ls
+  select key, value from char_serde_lb;
+select * from char_serde_ls limit 5;
+select value, count(*) from char_serde_ls group by value limit 5;
+
+--
+-- Columnar
+--
+create table  char_serde_c (
+  key char(15),
+  value char(20)
+) stored as rcfile;
+alter table char_serde_c set serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe';
+
+insert overwrite table char_serde_c
+  select key, value from char_serde_ls;
+select * from char_serde_c limit 5;
+select value, count(*) from char_serde_c group by value limit 5;
+
+--
+-- LazyBinaryColumnar
+--
+create table char_serde_lbc (
+  key char(15),
+  value char(20)
+) stored as rcfile;
+alter table char_serde_lbc set serde 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe';
+
+insert overwrite table char_serde_lbc
+  select key, value from char_serde_c;
+select * from char_serde_lbc limit 5;
+select value, count(*) from char_serde_lbc group by value limit 5;
+
+--
+-- ORC
+--
+create table char_serde_orc (
+  key char(15),
+  value char(20)
+) stored as orc;
+alter table char_serde_orc set serde 'org.apache.hadoop.hive.ql.io.orc.OrcSerde';
+
+
+insert overwrite table char_serde_orc
+  select key, value from char_serde_lbc;
+select * from char_serde_orc limit 5;
+select value, count(*) from char_serde_orc group by value limit 5;
+
+drop table if exists char_serde_regex;
+drop table if exists char_serde_lb;
+drop table if exists char_serde_ls;
+drop table if exists char_serde_c;
+drop table if exists char_serde_lbc;
+drop table if exists char_serde_orc;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_udf1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_udf1.q
new file mode 100644
index 0000000000000000000000000000000000000000..629d41dca9870088a544eaa2a836b1a1df1c7c04
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_udf1.q
@@ -0,0 +1,156 @@
+drop table char_udf_1;
+
+create table char_udf_1 (c1 string, c2 string, c3 char(10), c4 char(20));
+insert overwrite table char_udf_1
+  select key, value, key, value from src where key = '238' limit 1;
+
+-- UDFs with char support
+select 
+  concat(c1, c2),
+  concat(c3, c4),
+  concat(c1, c2) = concat(c3, c4)
+from char_udf_1 limit 1;
+
+select
+  upper(c2),
+  upper(c4),
+  upper(c2) = upper(c4)
+from char_udf_1 limit 1;
+
+select
+  lower(c2),
+  lower(c4),
+  lower(c2) = lower(c4)
+from char_udf_1 limit 1;
+
+-- Scalar UDFs
+select
+  ascii(c2),
+  ascii(c4),
+  ascii(c2) = ascii(c4)
+from char_udf_1 limit 1;
+
+select 
+  concat_ws('|', c1, c2),
+  concat_ws('|', c3, c4),
+  concat_ws('|', c1, c2) = concat_ws('|', c3, c4)
+from char_udf_1 limit 1;
+
+select
+  decode(encode(c2, 'US-ASCII'), 'US-ASCII'),
+  decode(encode(c4, 'US-ASCII'), 'US-ASCII'),
+  decode(encode(c2, 'US-ASCII'), 'US-ASCII') = decode(encode(c4, 'US-ASCII'), 'US-ASCII')
+from char_udf_1 limit 1;
+
+select
+  instr(c2, '_'),
+  instr(c4, '_'),
+  instr(c2, '_') = instr(c4, '_')
+from char_udf_1 limit 1;
+
+select
+  length(c2),
+  length(c4),
+  length(c2) = length(c4)
+from char_udf_1 limit 1;
+
+select
+  locate('a', 'abcdabcd', 3),
+  locate(cast('a' as char(1)), cast('abcdabcd' as char(10)), 3),
+  locate('a', 'abcdabcd', 3) = locate(cast('a' as char(1)), cast('abcdabcd' as char(10)), 3)
+from char_udf_1 limit 1;
+
+select
+  lpad(c2, 15, ' '),
+  lpad(c4, 15, ' '),
+  lpad(c2, 15, ' ') = lpad(c4, 15, ' ')
+from char_udf_1 limit 1;
+
+select
+  ltrim(c2),
+  ltrim(c4),
+  ltrim(c2) = ltrim(c4)
+from char_udf_1 limit 1;
+
+select
+  regexp(c2, 'val'),
+  regexp(c4, 'val'),
+  regexp(c2, 'val') = regexp(c4, 'val')
+from char_udf_1 limit 1;
+
+select
+  regexp_extract(c2, 'val_([0-9]+)', 1),
+  regexp_extract(c4, 'val_([0-9]+)', 1),
+  regexp_extract(c2, 'val_([0-9]+)', 1) = regexp_extract(c4, 'val_([0-9]+)', 1)
+from char_udf_1 limit 1;
+
+select
+  regexp_replace(c2, 'val', 'replaced'),
+  regexp_replace(c4, 'val', 'replaced'),
+  regexp_replace(c2, 'val', 'replaced') = regexp_replace(c4, 'val', 'replaced')
+from char_udf_1 limit 1;
+
+select
+  reverse(c2),
+  reverse(c4),
+  reverse(c2) = reverse(c4)
+from char_udf_1 limit 1;
+
+select
+  rpad(c2, 15, ' '),
+  rpad(c4, 15, ' '),
+  rpad(c2, 15, ' ') = rpad(c4, 15, ' ')
+from char_udf_1 limit 1;
+
+select
+  rtrim(c2),
+  rtrim(c4),
+  rtrim(c2) = rtrim(c4)
+from char_udf_1 limit 1;
+
+select
+  sentences('See spot run.  See jane run.'),
+  sentences(cast('See spot run.  See jane run.' as char(50)))
+from char_udf_1 limit 1;
+
+select
+  split(c2, '_'),
+  split(c4, '_')
+from char_udf_1 limit 1;
+
+select 
+  str_to_map('a:1,b:2,c:3',',',':'),
+  str_to_map(cast('a:1,b:2,c:3' as char(20)),',',':')
+from char_udf_1 limit 1;
+
+select
+  substr(c2, 1, 3),
+  substr(c4, 1, 3),
+  substr(c2, 1, 3) = substr(c4, 1, 3)
+from char_udf_1 limit 1;
+
+select
+  trim(c2),
+  trim(c4),
+  trim(c2) = trim(c4)
+from char_udf_1 limit 1;
+
+
+-- Aggregate Functions
+select
+  compute_stats(c2, 16),
+  compute_stats(c4, 16)
+from char_udf_1;
+
+select
+  min(c2),
+  min(c4)
+from char_udf_1;
+
+select
+  max(c2),
+  max(c4)
+from char_udf_1;
+
+
+drop table char_udf_1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_union1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_union1.q
new file mode 100644
index 0000000000000000000000000000000000000000..2ce5e89c2aa9e867e2824ab743060513f6ca75d9
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_union1.q
@@ -0,0 +1,47 @@
+drop table char_union1_ch1;
+drop table char_union1_ch2;
+drop table char_union1_str;
+
+create table  char_union1_ch1 (
+  c1 int,
+  c2 char(10)
+);
+
+create table  char_union1_ch2 (
+  c1 int,
+  c2 char(20)
+);
+
+create table  char_union1_str (
+  c1 int,
+  c2 string
+);
+
+load data local inpath '../../data/files/vc1.txt' into table char_union1_ch1;
+load data local inpath '../../data/files/vc1.txt' into table char_union1_ch2;
+load data local inpath '../../data/files/vc1.txt' into table char_union1_str;
+
+-- union char with same length char
+select * from (
+  select * from char_union1_ch1
+  union all
+  select * from char_union1_ch1 limit 1
+) q1 sort by c1;
+
+-- union char with different length char
+select * from (
+  select * from char_union1_ch1
+  union all
+  select * from char_union1_ch2 limit 1
+) q1 sort by c1;
+
+-- union char with string
+select * from (
+  select * from char_union1_ch1
+  union all
+  select * from char_union1_str limit 1
+) q1 sort by c1;
+
+drop table char_union1_ch1;
+drop table char_union1_ch2;
+drop table char_union1_str;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_varchar_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_varchar_udf.q
new file mode 100644
index 0000000000000000000000000000000000000000..332b84087e8efaad8f7a1dbc6d9ba4560da48947
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_varchar_udf.q
@@ -0,0 +1,9 @@
+DROP TABLE IF EXISTS  char_varchar_udf;
+
+CREATE TABLE char_varchar_udf (c char(8), vc varchar(10)) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t';
+LOAD DATA LOCAL INPATH '../../data/files/char_varchar_udf.txt' INTO TABLE char_varchar_udf;
+
+SELECT ROUND(c, 2), ROUND(vc, 3) FROM char_varchar_udf;
+SELECT AVG(c), AVG(vc), SUM(c), SUM(vc) FROM char_varchar_udf;
+
+DROP TABLE char_varchar_udf;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q
index 3c8a309991d750f941be2df3e665a12755d4f585..fbf8bba56e52c29de45a756ce1cf11e6c51abb92 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q
@@ -4,7 +4,7 @@ SET hive.stats.collect.scancols=true;
 -- This test is used for testing the ColumnAccessAnalyzer
 
 CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE;
 CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q
index 98627776d3163cdfdffcb1ceefef45d179e3574d..9dfe8ffbc3058cef3fb87567ebcaf5303279e029 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q
@@ -4,8 +4,8 @@ DROP TABLE Employee_Part;
 CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double)
 row format delimited fields terminated by '|'  stored as textfile;
 
-LOAD DATA LOCAL INPATH "../data/files/employee.dat" INTO TABLE Employee_Part partition(employeeSalary=2000.0);
-LOAD DATA LOCAL INPATH "../data/files/employee.dat" INTO TABLE Employee_Part partition(employeeSalary=4000.0);
+LOAD DATA LOCAL INPATH "../../data/files/employee.dat" INTO TABLE Employee_Part partition(employeeSalary=2000.0);
+LOAD DATA LOCAL INPATH "../../data/files/employee.dat" INTO TABLE Employee_Part partition(employeeSalary=4000.0);
 
 explain 
 analyze table Employee_Part partition (employeeSalary=2000.0) compute statistics for columns employeeID;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_tbllvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_tbllvl.q
index 72d88a67b521233f2268a033221e1842c5c8662e..170fbc51916a08efedc03fcfa3580613ea7774b4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_tbllvl.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_tbllvl.q
@@ -13,7 +13,7 @@ CREATE TABLE UserVisits_web_text_none (
   avgTimeOnSite int)
 row format delimited fields terminated by '|'  stored as textfile;
 
-LOAD DATA LOCAL INPATH "../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none;
+LOAD DATA LOCAL INPATH "../../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none;
 
 explain 
 analyze table UserVisits_web_text_none compute statistics for columns sourceIP, avgTimeOnSite, adRevenue;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compile_processor.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compile_processor.q
new file mode 100644
index 0000000000000000000000000000000000000000..6be02ec8a7bbbb1a5cf39e2bb0d15877be0bcf34
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compile_processor.q
@@ -0,0 +1,12 @@
+
+compile `import org.apache.hadoop.hive.ql.exec.UDF \;
+public class Pyth extends UDF {
+  public double evaluate(double a, double b){
+    return Math.sqrt((a*a) + (b*b)) \;
+  }
+} ` AS GROOVY NAMED Pyth.groovy;
+CREATE TEMPORARY FUNCTION Pyth as 'Pyth';
+
+SELECT Pyth(3,4) FROM src tablesample (1 rows);
+
+DROP TEMPORARY FUNCTION Pyth;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q
index c19813602782fb389bf32d0d8c29f59433c3ae0d..fd15634f202f75f59e50801e9a04cf6d61c2b3e1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q
@@ -1,7 +1,7 @@
 create table tab_binary(a binary);
 
 -- insert some data
-LOAD DATA LOCAL INPATH "../data/files/binary.txt" INTO TABLE tab_binary;
+LOAD DATA LOCAL INPATH "../../data/files/binary.txt" INTO TABLE tab_binary;
 
 select count(*) from tab_binary;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q
index dc76f7c7d57e12aff6b1c480f0e32610190d9dd2..cddb53f8f64f50e5988085ad22e9f86de441bb56 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q
@@ -1,7 +1,7 @@
 create table tab_bool(a boolean);
 
 -- insert some data
-LOAD DATA LOCAL INPATH "../data/files/bool.txt" INTO TABLE tab_bool;
+LOAD DATA LOCAL INPATH "../../data/files/bool.txt" INTO TABLE tab_bool;
 
 select count(*) from tab_bool;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_decimal.q
new file mode 100644
index 0000000000000000000000000000000000000000..77ec066dad985101bd44ac067ebb3a715ce4cb91
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_decimal.q
@@ -0,0 +1,11 @@
+set hive.stats.autogather=true;
+
+create table tab_decimal(a decimal(10,3));
+
+-- insert some data
+LOAD DATA LOCAL INPATH "../../data/files/decimal.txt" INTO TABLE tab_decimal;
+
+select count(*) from tab_decimal;
+
+-- compute statistical summary of data
+select compute_stats(a, 18) from tab_decimal;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q
index 6c6dc47e6770108b3c8d119ab2ec608dc0eac7c2..7a1e0f6295dc481c3b12c35519080d27387d73ea 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q
@@ -1,7 +1,7 @@
 create table tab_double(a double);
 
 -- insert some data
-LOAD DATA LOCAL INPATH "../data/files/double.txt" INTO TABLE tab_double;
+LOAD DATA LOCAL INPATH "../../data/files/double.txt" INTO TABLE tab_double;
 
 select count(*) from tab_double;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q
index 7d0a1584a28cc7374a0ebf8707e8df5b608b38b6..6a2070f7808f549f9825072b8e20d28a907ef01a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q
@@ -1,7 +1,7 @@
 create table tab_int(a int);
 
 -- insert some data
-LOAD DATA LOCAL INPATH "../data/files/int.txt" INTO TABLE tab_int;
+LOAD DATA LOCAL INPATH "../../data/files/int.txt" INTO TABLE tab_int;
 
 select count(*) from tab_int;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q
index f146f6b95b5449473d5ac493c049a3bcad839a5b..0023e7f6bd8184a82f17a880b5227f9bde012add 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q
@@ -1,7 +1,7 @@
 create table tab_string(a string);
 
 -- insert some data
-LOAD DATA LOCAL INPATH "../data/files/string.txt" INTO TABLE tab_string;
+LOAD DATA LOCAL INPATH "../../data/files/string.txt" INTO TABLE tab_string;
 
 select count(*) from tab_string;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q
index ced72d60445c22b5936d89f7b89a403b1b541a31..d51b80194e90aae147a75388726e1cb58072e1c4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 EXPLAIN
 SELECT NAMED_STRUCT(
          IF(ARRAY_CONTAINS(ARRAY(1, 2), 3), "F1", "B1"), 1,
@@ -7,7 +9,7 @@ SELECT NAMED_STRUCT(
          IF(ARRAY_CONTAINS(ARRAY(1, 2), 3), "F1", "B1"), 1,
          IF(ARRAY_CONTAINS(MAP_KEYS(MAP("b", "x")), "b"), "F2", "B2"), 2   
        ).F2
-       FROM src LIMIT 1;
+       FROM src tablesample (1 rows);
 
 SELECT NAMED_STRUCT(
          IF(ARRAY_CONTAINS(ARRAY(1, 2), 3), "F1", "B1"), 1,
@@ -17,4 +19,4 @@ SELECT NAMED_STRUCT(
          IF(ARRAY_CONTAINS(ARRAY(1, 2), 3), "F1", "B1"), 1,
          IF(ARRAY_CONTAINS(MAP_KEYS(MAP("b", "x")), "b"), "F2", "B2"), 2   
        ).F2
-       FROM src LIMIT 1;
+       FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q
index b3fd3f760eaf0bdc5923f11dc0f355068dad8283..0596f965ed89c27fe3823f7ec1f94a65fc0e01de 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q
@@ -104,7 +104,7 @@ FROM (SELECT x.key AS key, count(1) AS cnt
       
 set hive.optimize.correlation=false;
 -- If the key of a GroupByOperator is the right table's key in
--- a Left Outer Join, we cannot use a single MR to execute these two 
+-- a Left Outer Join, we cannot use a single MR to execute these two
 -- operators because those keys with a null value are not grouped.
 EXPLAIN
 SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt))
@@ -129,6 +129,29 @@ FROM (SELECT y.key AS key, count(1) AS cnt
       FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key)
       GROUP BY y.key) tmp;
 
+set hive.optimize.correlation=false;
+-- If a column of the key of a GroupByOperator is the right table's key in
+-- a Left Outer Join, we cannot use a single MR to execute these two
+-- operators because those keys with a null value are not grouped.
+EXPLAIN
+SELECT x.key, y.value, count(1) AS cnt
+FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key AND x.value = y.value)
+GROUP BY x.key, y.value;
+
+SELECT x.key, y.value, count(1) AS cnt
+FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key AND x.value = y.value)
+GROUP BY x.key, y.value;
+
+set hive.optimize.correlation=true;
+EXPLAIN
+SELECT x.key, y.value, count(1) AS cnt
+FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key AND x.value = y.value)
+GROUP BY x.key, y.value;
+
+SELECT x.key, y.value, count(1) AS cnt
+FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key AND x.value = y.value)
+GROUP BY x.key, y.value;
+
 set hive.optimize.correlation=false;
 -- If the key of a GroupByOperator is the right table's key in
 -- a Right Outer Join, these two operators will be executed in
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q
index 70fcdfc0b4288d4b37f242357842e8b03e4a3e8e..953d191fc608798260898c6f11b038e86176b9f3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q
@@ -1,9 +1,9 @@
 CREATE TABLE T1(key INT, val STRING);
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 CREATE TABLE T2(key INT, val STRING);
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 CREATE TABLE T3(key INT, val STRING);
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T3;
 
 set hive.auto.convert.join=false;
 set hive.optimize.correlation=false;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q
index ac836c0bfdc916cf572bfb9771b3f89be1bb3782..287c7a389d1a83d7564d2ac8688cc245e3de0b23 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q
@@ -1,11 +1,11 @@
 CREATE TABLE T1(key INT, val STRING);
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T1;
 CREATE TABLE T2(key INT, val STRING);
-LOAD DATA LOCAL INPATH '../data/files/kv2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/kv2.txt' INTO TABLE T2;
 CREATE TABLE T3(key INT, val STRING);
-LOAD DATA LOCAL INPATH '../data/files/kv3.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/kv3.txt' INTO TABLE T3;
 CREATE TABLE T4(key INT, val STRING);
-LOAD DATA LOCAL INPATH '../data/files/kv5.txt' INTO TABLE T4;
+LOAD DATA LOCAL INPATH '../../data/files/kv5.txt' INTO TABLE T4;
 
 CREATE TABLE dest_co1(key INT, val STRING);
 CREATE TABLE dest_co2(key INT, val STRING);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q
index 0d66a5ec345e698108ec63cc0712f4a14b0af9d0..74ae9e428befe2aaac28388c03a490475910e82e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q
@@ -1,5 +1,5 @@
 create table abcd (a int, b int, c int, d int);
-LOAD DATA LOCAL INPATH '../data/files/in4.txt' INTO TABLE abcd;
+LOAD DATA LOCAL INPATH '../../data/files/in4.txt' INTO TABLE abcd;
 
 select * from abcd;
 set hive.map.aggr=true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_func1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_func1.q
new file mode 100644
index 0000000000000000000000000000000000000000..ad924d345304c34d07f6fa0afdac0c2ef56aa43d
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_func1.q
@@ -0,0 +1,30 @@
+
+-- qtest_get_java_boolean should already be created during test initialization
+select qtest_get_java_boolean('true'), qtest_get_java_boolean('false') from src limit 1;
+
+create database mydb;
+create function mydb.func1 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper';
+
+show functions mydb.func1;
+
+select mydb.func1('abc') from src limit 1;
+
+drop function mydb.func1;
+
+-- function should now be gone
+show functions mydb.func1;
+
+-- To test function name resolution
+create function mydb.qtest_get_java_boolean as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper';
+
+use default;
+-- unqualified function should resolve to one in default db
+select qtest_get_java_boolean('abc'), default.qtest_get_java_boolean('abc'), mydb.qtest_get_java_boolean('abc') from default.src limit 1;
+
+use mydb;
+-- unqualified function should resolve to one in mydb db
+select qtest_get_java_boolean('abc'), default.qtest_get_java_boolean('abc'), mydb.qtest_get_java_boolean('abc') from default.src limit 1;
+
+drop function mydb.qtest_get_java_boolean;
+
+drop database mydb cascade;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q
index cb4d6578af476d51363a6a118c750ae6f1d3ab03..13539a65f512f4b1702b0dc87d1df7b89fb45223 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q
@@ -21,8 +21,8 @@ INSERT OVERWRITE TABLE table2 SELECT key, value FROM src WHERE key = 100;
 SELECT * FROM table1;
 SELECT * FROM table2;
 
-CREATE EXTERNAL TABLE table4 (a INT) LOCATION '${system:test.src.data.dir}/files/ext_test';
-CREATE EXTERNAL TABLE table5 LIKE table4 LOCATION '${system:test.src.data.dir}/files/ext_test';
+CREATE EXTERNAL TABLE table4 (a INT) LOCATION '${system:hive.root}/data/files/ext_test';
+CREATE EXTERNAL TABLE table5 LIKE table4 LOCATION '${system:hive.root}/data/files/ext_test';
 
 SELECT * FROM table4;
 SELECT * FROM table5;
@@ -31,5 +31,5 @@ DROP TABLE table5;
 SELECT * FROM table4;
 DROP TABLE table4;
 
-CREATE EXTERNAL TABLE table4 (a INT) LOCATION '${system:test.src.data.dir}/files/ext_test';
+CREATE EXTERNAL TABLE table4 (a INT) LOCATION '${system:hive.root}/data/files/ext_test';
 SELECT * FROM table4;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q
index 4418b3430affc6fbb6349cdbf939917251709e2d..483931b6fff4ba0a483f65aa9e355588afcf773a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q
@@ -1,6 +1,6 @@
 create table src_rc_merge_test(key int, value string) stored as rcfile;
 
-load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test;
+load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test;
 
 set hive.exec.compress.output = true;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q
index 2debd0d71d224e6395d08fe1e0f5c77a70ed86b6..735b139719700a6f9f069463d1231dde3e97c6af 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q
@@ -9,7 +9,7 @@ CREATE TABLE table1 (
 DESCRIBE table1;
 DESCRIBE EXTENDED table1;
 
-LOAD DATA LOCAL INPATH '../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table1;
+LOAD DATA LOCAL INPATH '../../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table1;
 
 SELECT * from table1;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q
index dd5aa63e4578af5400519c364dc5e7bf7220b482..1e5d151f286ba593e46c254a65407fc616cd318e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q
@@ -4,7 +4,7 @@ row format delimited
   fields terminated by '\t'
   collection items terminated by '\001';
 
-load data local inpath '../data/files/kv1.txt'
+load data local inpath '../../data/files/kv1.txt'
 overwrite into table abc;
 
 SELECT strct, strct.a, strct.b FROM abc LIMIT 10;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q
index bb0e5b989d54dcb52b9c4ca7d0fe318d7bec8a77..6bc4d29358c11f5fe00dc43e829169694adeff78 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q
@@ -4,7 +4,7 @@ strct struct<a:int, b:string, c:string>);
 create table abc(mydata uniontype<int,double,array<string>,struct<a:int,b:string>>,
 strct struct<a:int, b:string, c:string>);
 
-load data local inpath '../data/files/union_input.txt'
+load data local inpath '../../data/files/union_input.txt'
 overwrite into table abc;
 
 SELECT * FROM abc;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q
index 2199750f42f7e2b3707923b02195900116ae7843..11ba9c8afc61dd358d72e7801b83f4d5000b5d1a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q
@@ -11,3 +11,14 @@ describe formatted w;
 
 drop view v;
 drop view w;
+
+
+-- HIVE-4116 Can't use views using map datatype.
+
+CREATE TABLE items (id INT, name STRING, info MAP<STRING,STRING>);
+
+explain
+CREATE VIEW priceview AS SELECT items.id, items.info['price'] FROM items;
+CREATE VIEW priceview AS SELECT items.id, items.info['price'] FROM items;
+
+select * from priceview;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_1.q
new file mode 100644
index 0000000000000000000000000000000000000000..fb38c947b32d4120d7b83e46e2b672302ee44134
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_1.q
@@ -0,0 +1,26 @@
+
+create table A as
+select * from src;
+
+create table B as
+select * from src
+limit 10;
+
+set hive.auto.convert.join.noconditionaltask.size=100;
+
+explain select * from A join B;
+
+explain select * from B d1 join B d2 on d1.key = d2.key join A;
+
+explain select * from A join 
+         (select d1.key 
+          from B d1 join B d2 on d1.key = d2.key 
+          where 1 = 1 group by d1.key) od1;
+          
+explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1;
+
+explain select * from 
+(select A.key from A group by key) ss join 
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1;
+
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_2.q
new file mode 100644
index 0000000000000000000000000000000000000000..479d57137e94664da0e264ec4e3607c730a0e2a0
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_2.q
@@ -0,0 +1,27 @@
+create table A as
+select * from src;
+
+create table B as
+select * from src
+limit 10;
+
+set hive.auto.convert.join=true;
+set hive.auto.convert.join.noconditionaltask=true;
+set hive.auto.convert.join.noconditionaltask.size=10000000;
+
+explain select * from A join B;
+
+explain select * from B d1 join B d2 on d1.key = d2.key join A;
+
+explain select * from A join 
+         (select d1.key 
+          from B d1 join B d2 on d1.key = d2.key 
+          where 1 = 1 group by d1.key) od1;
+          
+explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1;
+
+explain select * from 
+(select A.key from A group by key) ss join 
+(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1;
+
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q
index e595904b412fa0f14190deedf7fecd73e12c6316..71af40e7e41403b7aef547f87b19f1aca659964a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q
@@ -56,7 +56,7 @@ set hive.exec.mode.local.auto=true;
 create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10;
 
 create table nzhang_ctas6 (key string, `to` string);
-insert overwrite table nzhang_ctas6 select key, value from src limit 10;
+insert overwrite table nzhang_ctas6 select key, value from src tablesample (10 rows);
 create table nzhang_ctas7 as select key, `to` from nzhang_ctas6;
 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_char.q
new file mode 100644
index 0000000000000000000000000000000000000000..ecfe74afd02733b447d1c42ae52af99a9bd415e7
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_char.q
@@ -0,0 +1,22 @@
+drop table ctas_char_1;
+drop table ctas_char_2;
+drop view ctas_char_3;
+
+create table ctas_char_1 (key char(10), value string);
+insert overwrite table ctas_char_1 
+  select key, value from src sort by key, value limit 5;
+
+-- create table as with char column
+create table ctas_char_2 as select key, value from ctas_char_1;
+
+-- view with char column
+create view ctas_char_3 as select key, value from ctas_char_2;
+
+select key, value from ctas_char_1;
+select * from ctas_char_2;
+select * from ctas_char_3;
+
+
+drop table ctas_char_1;
+drop table ctas_char_2;
+drop view ctas_char_3;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q
index 4961b971dbdec497fe8d614b0c067fe1f98a26ff..f39689de03a55a544ca7de4d24184d0dcf999308 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q
@@ -58,11 +58,6 @@ create table nzhang_ctas6 (key string, `to` string);
 insert overwrite table nzhang_ctas6 select key, value from src limit 10;
 create table nzhang_ctas7 as select key, `to` from nzhang_ctas6;
 
-
-
-
-
-
-
-
-
+create table nzhang_ctas8 as select 3.14BD from nzhang_ctas6 limit 1;
+desc nzhang_ctas8;
+drop table nzhang_ctas8;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_1.q
new file mode 100644
index 0000000000000000000000000000000000000000..ca4132f7d3fb93fa219a6711916d81dd2b056fd8
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_1.q
@@ -0,0 +1,28 @@
+explain
+with q1 as ( select key from src where key = '5')
+select *
+from q1
+;
+
+with q1 as ( select key from src where key = '5')
+select *
+from q1
+;
+
+-- in subquery
+explain
+with q1 as ( select key from src where key = '5')
+select * from (select key from q1) a;
+
+with q1 as ( select key from src where key = '5')
+select * from (select key from q1) a;
+
+-- chaining
+explain
+with q1 as ( select key from q2 where key = '5'),
+q2 as ( select key from src where key = '5')
+select * from (select key from q1) a;
+
+with q1 as ( select key from q2 where key = '5'),
+q2 as ( select key from src where key = '5')
+select * from (select key from q1) a;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_2.q
new file mode 100644
index 0000000000000000000000000000000000000000..b49620aca144df2aea41d3935accfa79b0eed4d9
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_2.q
@@ -0,0 +1,56 @@
+
+-- union test
+with q1 as (select * from src where key= '5'),
+q2 as (select * from src s2 where key = '4')
+select * from q1 union all select * from q2
+;
+
+-- insert test
+create table s1 like src;
+with q1 as ( select key, value from src where key = '5')
+from q1
+insert overwrite table s1
+select *
+;
+select * from s1;
+drop table s1;
+
+-- from style
+with q1 as (select * from src where key= '5')
+from q1
+select *
+;
+
+-- ctas
+create table s2 as
+with q1 as ( select key from src where key = '4')
+select * from q1
+;
+
+select * from s2;
+drop table s2;
+
+-- view test
+create view v1 as
+with q1 as ( select key from src where key = '5')
+select * from q1
+;
+
+select * from v1;
+
+drop view v1;
+
+
+-- view test, name collision
+create view v1 as
+with q1 as ( select key from src where key = '5')
+select * from q1
+;
+
+with q1 as ( select key from src where key = '4')
+select * from v1
+;
+
+drop view v1;
+
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q
index ff5e86dc5a6b4ca5cc8449a1107a5e915a6a9bc5..d769d05602a177366426e8bc33563caf4c525e43 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q
@@ -1,6 +1,7 @@
-ADD JAR ../build/ql/test/test-udfs.jar;
 CREATE TABLE src1_rot13_iof(key STRING, value STRING) 
   STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.udf.Rot13InputFormat'
             OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.udf.Rot13OutputFormat';
+DESCRIBE EXTENDED src1_rot13_iof;
+SELECT * FROM src1 ORDER BY key, value;
 INSERT OVERWRITE TABLE src1_rot13_iof SELECT * FROM src1;
-SELECT * FROM src1_rot13_iof;
+SELECT * FROM src1_rot13_iof ORDER BY key, value;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q
index 9140a42b650e8b34df665941c4696b0f1c1b3e7e..e3ceaccfa5ef6c4d5082b550cdb2bca4b6811384 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q
@@ -58,7 +58,7 @@ SHOW TABLES;
 DESCRIBE EXTENDED test_table_like;
 
 -- LOAD and SELECT
-LOAD DATA LOCAL INPATH '../data/files/test.dat'
+LOAD DATA LOCAL INPATH '../../data/files/test.dat'
 OVERWRITE INTO TABLE test_table;
 SELECT * FROM test_table;
 
@@ -146,7 +146,7 @@ CREATE TABLE db1.src(key STRING, value STRING)
 STORED AS TEXTFILE;
 
 -- LOAD into foreign table
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt'
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt'
 OVERWRITE INTO TABLE db1.src;
 
 -- SELECT from foreign table
@@ -158,7 +158,7 @@ PARTITIONED BY (ds STRING, hr STRING)
 STORED AS TEXTFILE;
 
 -- LOAD data into Partitioned foreign table
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt'
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt'
 OVERWRITE INTO TABLE db1.srcpart
 PARTITION (ds='2008-04-08', hr='11');
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q
index 4e17c7ad5ac105ba249e71ec63728fc7d3f21dd0..13712732452153aac3884429360ffadc2b233d56 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q
@@ -8,13 +8,14 @@ CREATE DATABASE db5;
 SHOW DATABASES;
 USE db5;
 
+set hive.stats.dbclass=fs;
 dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/temp;
 dfs -rmr ${system:test.tmp.dir}/dbcascade;
 dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade;
 
 -- add a table, index and view
 CREATE TABLE temp_tbl (id INT, name STRING);
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE temp_tbl;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE temp_tbl;
 CREATE VIEW temp_tbl_view AS SELECT * FROM temp_tbl;
 CREATE INDEX idx1 ON TABLE temp_tbl(id) AS 'COMPACT' with DEFERRED REBUILD;
 ALTER INDEX idx1 ON temp_tbl REBUILD;
@@ -23,15 +24,15 @@ dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/temp_tbl2;
 dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/temp_tbl2_idx2;
 -- add a table, index and view with a different storage location
 CREATE TABLE temp_tbl2 (id INT, name STRING) LOCATION 'file:${system:test.tmp.dir}/dbcascade/temp_tbl2';
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' into table temp_tbl2;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' into table temp_tbl2;
 CREATE VIEW temp_tbl2_view AS SELECT * FROM temp_tbl2;
 CREATE INDEX idx2 ON TABLE temp_tbl2(id) AS 'COMPACT' with DEFERRED REBUILD LOCATION 'file:${system:test.tmp.dir}/dbcascade/temp_tbl2_idx2';
 ALTER INDEX idx2 ON temp_tbl2 REBUILD;
 
 -- add a partitioned table, index and view
 CREATE TABLE part_tab (id INT, name STRING)  PARTITIONED BY (ds string);
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab PARTITION (ds='2008-04-09');
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab PARTITION (ds='2009-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab PARTITION (ds='2008-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab PARTITION (ds='2009-04-09');
 CREATE INDEX idx3 ON TABLE part_tab(id) AS 'COMPACT' with DEFERRED REBUILD;
 ALTER INDEX idx3 ON part_tab PARTITION (ds='2008-04-09') REBUILD;
 ALTER INDEX idx3 ON part_tab PARTITION (ds='2009-04-09') REBUILD;
@@ -41,8 +42,8 @@ dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/part_tab2_idx4;
 -- add a partitioned table, index and view with a different storage location
 CREATE TABLE part_tab2 (id INT, name STRING)  PARTITIONED BY (ds string)
 		LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab2';
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab2 PARTITION (ds='2008-04-09');
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab2 PARTITION (ds='2009-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab2 PARTITION (ds='2008-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab2 PARTITION (ds='2009-04-09');
 CREATE INDEX idx4 ON TABLE part_tab2(id) AS 'COMPACT' with DEFERRED REBUILD
 		LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab2_idx4';
 ALTER INDEX idx4 ON part_tab2 PARTITION (ds='2008-04-09') REBUILD;
@@ -56,8 +57,8 @@ dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/part_tab3_idx5;
 CREATE TABLE part_tab3 (id INT, name STRING)  PARTITIONED BY (ds string)
 		LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab3';
 ALTER TABLE part_tab3 ADD PARTITION  (ds='2007-04-09') LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab3_p1';
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab3 PARTITION (ds='2008-04-09');
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab3 PARTITION (ds='2009-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab3 PARTITION (ds='2008-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab3 PARTITION (ds='2009-04-09');
 CREATE INDEX idx5 ON TABLE part_tab3(id) AS 'COMPACT' with DEFERRED REBUILD
 		LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab3_idx5';
 ALTER INDEX idx5 ON part_tab3 PARTITION (ds='2008-04-09') REBUILD;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q
index a2322fc1e45a015d98bcc0e360c9bdb9112c29c4..7d89ac9268f9b3277d23b6bfda703d06cd28cc4f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q
@@ -1,21 +1,23 @@
+set hive.fetch.task.conversion=more;
+
 drop table date_1;
 
 create table date_1 (d date);
 
 insert overwrite table date_1 
-  select cast('2011-01-01' as date) from src limit 1;
+  select cast('2011-01-01' as date) from src tablesample (1 rows);
 
 select * from date_1 limit 1;
 select d, count(d) from date_1 group by d;
 
 insert overwrite table date_1 
-  select date '2011-01-01' from src limit 1;
+  select date '2011-01-01' from src tablesample (1 rows);
 
 select * from date_1 limit 1;
 select d, count(d) from date_1 group by d;
 
 insert overwrite table date_1 
-  select cast(cast('2011-01-01 00:00:00' as timestamp) as date) from src limit 1;
+  select cast(cast('2011-01-01 00:00:00' as timestamp) as date) from src tablesample (1 rows);
 
 select * from date_1 limit 1;
 select d, count(d) from date_1 group by d;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q
index 0821e012ba26b43090c5109c4598dc008dd55a04..c5346c87dd25cf7e4edfe21fac9514d0c0322fe5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q
@@ -8,7 +8,7 @@ create table date_2 (
   FL_NUM int
 );
 
-LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_2;
+LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_2;
 
 select fl_date, fl_num from date_2 order by fl_date asc, fl_num desc;
 select fl_date, fl_num from date_2 order by fl_date desc, fl_num asc;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q
index be251484459b6c1dc2ced833652bcad8610f5f5f..383fb4e9904da58a1a6fafe6bc955dd5f6f0a07d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q
@@ -7,7 +7,7 @@ create table date_3 (
 alter table date_3 add columns (c2 date);
 
 insert overwrite table date_3
-  select 1, cast(cast('2011-01-01 00:00:00' as timestamp) as date) from src limit 1;
+  select 1, cast(cast('2011-01-01 00:00:00' as timestamp) as date) from src tablesample (1 rows);
 
 select * from date_3;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q
index 4801a79b8795b4fa1c499336842dc262210251aa..c840089f2efeb082426184ede451c586a8217ae5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 drop table date_4;
 
 create table date_4 (d date);
@@ -5,7 +7,7 @@ alter table date_4 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
 -- Test date literal syntax
 insert overwrite table date_4 
-  select date '2011-01-01' from src limit 1;
+  select date '2011-01-01' from src tablesample (1 rows);
 select d, date '2011-01-01' from date_4 limit 1;
 
 drop table date_4;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q
index bdcb6c1b6e44738eae6d636392dbf7ef8b204e0c..86c7362e295f2246d6b59abb15f1d4a7b853c90b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 -- Comparisons against same value
 select cast('2011-05-06' as date) > 
   cast('2011-05-06' as date) from src limit 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q
index a5844b76e9ef89dcd8bb1d15615874e35a2713ab..34bb8c8990a6be689be41f7182d7d7d070cc010f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q
@@ -8,7 +8,7 @@ create table date_join1 (
   FL_NUM int
 );
 
-LOAD DATA LOCAL INPATH '../data/files/flights_join.txt' OVERWRITE INTO TABLE date_join1;
+LOAD DATA LOCAL INPATH '../../data/files/flights_join.txt' OVERWRITE INTO TABLE date_join1;
 
 -- Note that there are 2 rows with date 2000-11-28, so we should expect 4 rows with that date in the join results
 select t1.fl_num, t1.fl_date, t2.fl_num, t2.fl_date
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q
index ffc06d270d53ac38db212a0e54b451d4ecc450f7..24b48206802aeee0ceb39594e6e3eb1a520c52f6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q
@@ -22,7 +22,7 @@ with serdeproperties (
 )
 stored as textfile;
 
-load data local inpath '../data/files/flights_tiny.txt.1' overwrite into table date_serde_regex;
+load data local inpath '../../data/files/flights_tiny.txt.1' overwrite into table date_serde_regex;
 
 select * from date_serde_regex;
 select fl_date, count(*) from date_serde_regex group by fl_date;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q
index 9696320a85fcde2b23d6645596edca1ffa7a4e92..c55b9f9147adada69e0fd2977222d085fbf63e8d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q
@@ -17,7 +17,7 @@ create table date_udf_flight (
   ARR_DELAY float,
   FL_NUM int
 );
-LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_udf_flight;
+LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_udf_flight;
 
 -- Test UDFs with date input
 select unix_timestamp(d), year(d), month(d), day(d), dayofmonth(d),
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q
new file mode 100644
index 0000000000000000000000000000000000000000..6612fe8babdf8a0e73da8c191d132e1e5e69ce19
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q
@@ -0,0 +1,12 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.txn.testing=true;
+
+create table T1(key string, val string) stored as textfile;
+
+set hive.txn.testing=true;
+alter table T1 compact 'major';
+
+alter table T1 compact 'minor';
+
+drop table T1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q
new file mode 100644
index 0000000000000000000000000000000000000000..599cad9afc611cec533a78fc8120d22b721f881b
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q
@@ -0,0 +1,14 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.txn.testing=true;
+
+create table T1(key string, val string) partitioned by (ds string) stored as textfile;
+
+alter table T1 add partition (ds = 'today');
+alter table T1 add partition (ds = 'yesterday');
+
+alter table T1 partition (ds = 'today') compact 'major';
+
+alter table T1 partition (ds = 'yesterday') compact 'minor';
+
+drop table T1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q
new file mode 100644
index 0000000000000000000000000000000000000000..871d292a59ce3ee72eda6928143df8acae4bf092
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q
@@ -0,0 +1,15 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.txn.testing=true;
+
+create database D1;
+
+use D1;
+
+create table T1(key string, val string) stored as textfile;
+
+alter table T1 compact 'major';
+
+alter table T1 compact 'minor';
+
+drop table T1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_ddl1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_ddl1.q
new file mode 100644
index 0000000000000000000000000000000000000000..3126bd6e5460e13755b4f2b118d832142c959491
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_ddl1.q
@@ -0,0 +1,59 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+create database D1;
+
+alter database D1 set dbproperties('test'='yesthisis');
+
+drop database D1;
+
+create table T1(key string, val string) stored as textfile;
+
+create table T2 like T1;
+
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
+
+select * from T1;
+
+create table T3 as select * from T1;
+
+create table T4 (key char(10), val decimal(5,2), b int)
+    partitioned by (ds string)
+    clustered by (b) into 10 buckets
+    stored as orc;
+
+alter table T3 rename to newT3;
+
+alter table T2 set tblproperties ('test'='thisisatest');
+
+alter table T2 set serde 'org.apache.hadoop.hive.ql.io.orc.OrcSerde';
+alter table T2 set serdeproperties ('test'='thisisatest');
+
+alter table T2 clustered by (key) into 32 buckets;
+
+alter table T4 add partition (ds='today'); 
+
+alter table T4 partition (ds='today') rename to partition(ds='yesterday');
+
+alter table T4 drop partition (ds='yesterday');
+
+alter table T4 add partition (ds='tomorrow'); 
+
+create table T5 (a string, b int);
+alter table T5 set fileformat orc;
+
+create table T7 (a string, b int);
+alter table T7 set location 'file:///tmp';
+
+alter table T2 touch;
+alter table T4 touch partition (ds='tomorrow');
+
+create view V1 as select key from T1;
+alter view V1 set tblproperties ('test'='thisisatest');
+drop view V1;
+
+
+
+drop table T1;
+drop table T2;
+drop table newT3;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query1.q
new file mode 100644
index 0000000000000000000000000000000000000000..970069aca676689c9ee36b4b3dbce94e01993623
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query1.q
@@ -0,0 +1,17 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+create table T1(key string, val string) stored as textfile;
+
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
+
+select * from T1;
+
+create table T2(key string, val string) stored as textfile;
+
+insert into table T2 select * from T1;
+
+select * from T2;
+
+drop table T1;
+drop table T2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query2.q
new file mode 100644
index 0000000000000000000000000000000000000000..00942e5357892632cc913b707d228bdc63a25d39
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query2.q
@@ -0,0 +1,17 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+create table T1(key string, val string) stored as textfile;
+
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
+
+select * from T1;
+
+create table T2(key string, val string) stored as textfile;
+
+insert overwrite table T2 select * from T1;
+
+select * from T2;
+
+drop table T1;
+drop table T2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query3.q
new file mode 100644
index 0000000000000000000000000000000000000000..75b642b5492cce9d05801628a6122dcfab2de613
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query3.q
@@ -0,0 +1,21 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+create table T1(key string, val string) stored as textfile;
+
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
+
+select * from T1;
+
+create table T2(key string, val string) partitioned by (pval string) stored as textfile;
+
+insert into table T2 partition (pval = '1') select * from T1;
+
+select * from T2;
+
+insert overwrite table T2 partition (pval = '1') select * from T1;
+
+select * from T2;
+
+drop table T1;
+drop table T2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query4.q
new file mode 100644
index 0000000000000000000000000000000000000000..57eb4424ea5ccee585787096331c0f604038048a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query4.q
@@ -0,0 +1,19 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.exec.dynamic.partition.mode=nonstrict;
+set hive.exec.dynamic.partition=true;
+
+create table T1(key string, val string) stored as textfile;
+
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
+
+select * from T1;
+
+create table T2(key string) partitioned by (val string) stored as textfile;
+
+insert overwrite table T2 partition (val) select key, val from T1;
+
+select * from T2;
+
+drop table T1;
+drop table T2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query5.q
new file mode 100644
index 0000000000000000000000000000000000000000..d22b98fd7df2d449ebd8dc22a2850a49eef4ac66
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query5.q
@@ -0,0 +1,24 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+create database foo;
+
+use foo;
+
+create table T1(key string, val string) partitioned by (ds string) stored as textfile;
+
+alter table T1 add partition (ds='today');
+
+create view V1 as select key from T1;
+
+show tables;
+
+describe T1;
+
+drop view V1;
+
+drop table T1;
+
+show databases;
+
+drop database foo;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q
new file mode 100644
index 0000000000000000000000000000000000000000..7c71fdd9d230f5927a2fcb37181e610e7ec7ff9c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q
@@ -0,0 +1,11 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.txn.testing=true;
+
+show locks;
+
+show locks extended;
+
+show locks default;
+
+show transactions;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q
index 6c689e188a908080ffd35bfa950b0ea733337733..f52b1923eb06733b2748ce151eb3c110a664b4d7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q
@@ -1,18 +1,22 @@
-drop table decimal_1;
+set hive.fetch.task.conversion=more;
+
+drop table if exists decimal_1;
 
-create table decimal_1 (t decimal);
+create table decimal_1 (t decimal(4,2), u decimal(5), v decimal);
 alter table decimal_1 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe';
 
+desc decimal_1;
+
 insert overwrite table decimal_1
-  select cast('17.29' as decimal) from src limit 1;
-select cast(t as boolean) from decimal_1 limit 1;
-select cast(t as tinyint) from decimal_1 limit 1;
-select cast(t as smallint) from decimal_1 limit 1;
-select cast(t as int) from decimal_1 limit 1;
-select cast(t as bigint) from decimal_1 limit 1;
-select cast(t as float) from decimal_1 limit 1;
-select cast(t as double) from decimal_1 limit 1;
-select cast(t as string) from decimal_1 limit 1;
-select cast(t as timestamp) from decimal_1 limit 1;
+  select cast('17.29' as decimal(4,2)), 3.1415926BD, 3115926.54321BD from src tablesample (1 rows);
+select cast(t as boolean) from decimal_1;
+select cast(t as tinyint) from decimal_1;
+select cast(t as smallint) from decimal_1;
+select cast(t as int) from decimal_1;
+select cast(t as bigint) from decimal_1;
+select cast(t as float) from decimal_1;
+select cast(t as double) from decimal_1;
+select cast(t as string) from decimal_1;
+select cast(t as timestamp) from decimal_1;
 
 drop table decimal_1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q
index 4890618a0dc328b1bbb3fdbb94e9c3e79920766a..2c4d9190794232d3b775c89571fb72a67d15efec 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q
@@ -1,40 +1,42 @@
+set hive.fetch.task.conversion=more;
+
 drop table decimal_2;
 
-create table decimal_2 (t decimal);
+create table decimal_2 (t decimal(18,9));
 alter table decimal_2 set serde 'org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe';
 
 insert overwrite table decimal_2
-  select cast('17.29' as decimal) from src limit 1;
+  select cast('17.29' as decimal(4,2)) from src tablesample (1 rows);
 
-select cast(t as boolean) from decimal_2 limit 1;
-select cast(t as tinyint) from decimal_2 limit 1;
-select cast(t as smallint) from decimal_2 limit 1;
-select cast(t as int) from decimal_2 limit 1;
-select cast(t as bigint) from decimal_2 limit 1;
-select cast(t as float) from decimal_2 limit 1;
-select cast(t as double) from decimal_2 limit 1;
-select cast(t as string) from decimal_2 limit 1;
+select cast(t as boolean) from decimal_2;
+select cast(t as tinyint) from decimal_2;
+select cast(t as smallint) from decimal_2;
+select cast(t as int) from decimal_2;
+select cast(t as bigint) from decimal_2;
+select cast(t as float) from decimal_2;
+select cast(t as double) from decimal_2;
+select cast(t as string) from decimal_2;
 
 insert overwrite table decimal_2
-  select cast('3404045.5044003' as decimal) from src limit 1;
+  select cast('3404045.5044003' as decimal(18,9)) from src tablesample (1 rows);
 
-select cast(t as boolean) from decimal_2 limit 1;
-select cast(t as tinyint) from decimal_2 limit 1;
-select cast(t as smallint) from decimal_2 limit 1;
-select cast(t as int) from decimal_2 limit 1;
-select cast(t as bigint) from decimal_2 limit 1;
-select cast(t as float) from decimal_2 limit 1;
-select cast(t as double) from decimal_2 limit 1;
-select cast(t as string) from decimal_2 limit 1;
+select cast(t as boolean) from decimal_2;
+select cast(t as tinyint) from decimal_2;
+select cast(t as smallint) from decimal_2;
+select cast(t as int) from decimal_2;
+select cast(t as bigint) from decimal_2;
+select cast(t as float) from decimal_2;
+select cast(t as double) from decimal_2;
+select cast(t as string) from decimal_2;
 
-select cast(3.14 as decimal) from decimal_2 limit 1;
-select cast(cast(3.14 as float) as decimal) from decimal_2 limit 1;
-select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal) from decimal_2 limit 1;
-select cast(true as decimal) from decimal_2 limit 1;
-select cast(3Y as decimal) from decimal_2 limit 1;
-select cast(3S as decimal) from decimal_2 limit 1;
-select cast(cast(3 as int) as decimal) from decimal_2 limit 1;
-select cast(3L as decimal) from decimal_2 limit 1;
-select cast(0.99999999999999999999 as decimal) from decimal_2 limit 1;
-select cast('0.99999999999999999999' as decimal) from decimal_2 limit 1;
+select cast(3.14 as decimal(4,2)) from decimal_2;
+select cast(cast(3.14 as float) as decimal(4,2)) from decimal_2;
+select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal(30,8)) from decimal_2;
+select cast(true as decimal) from decimal_2;
+select cast(3Y as decimal) from decimal_2;
+select cast(3S as decimal) from decimal_2;
+select cast(cast(3 as int) as decimal) from decimal_2;
+select cast(3L as decimal) from decimal_2;
+select cast(0.99999999999999999999 as decimal(20,19)) from decimal_2;
+select cast('0.99999999999999999999' as decimal(20,20)) from decimal_2;
 drop table decimal_2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q
index 28211e3f14cc1c287d1e5e5af8823cfaa4e0d0f8..e4fba06fea1a9dc48868cfd98c7cb892e1e99d45 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q
@@ -1,11 +1,11 @@
 DROP TABLE IF EXISTS DECIMAL_3;
 
-CREATE TABLE DECIMAL_3(key decimal, value int) 
+CREATE TABLE DECIMAL_3(key decimal(38,18), value int)
 ROW FORMAT DELIMITED
    FIELDS TERMINATED BY ' '
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_3;
+LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_3;
 
 SELECT * FROM DECIMAL_3 ORDER BY key, value;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q
index e8a89c131cd85f3283fec90820d8f2e07db8ca2e..699ba3cb4f558e9436651ed1327f441de6c47099 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q
@@ -1,15 +1,15 @@
 DROP TABLE IF EXISTS DECIMAL_4_1;
 DROP TABLE IF EXISTS DECIMAL_4_2;
 
-CREATE TABLE DECIMAL_4_1(key decimal, value int) 
+CREATE TABLE DECIMAL_4_1(key decimal(35,25), value int) 
 ROW FORMAT DELIMITED
    FIELDS TERMINATED BY ' '
 STORED AS TEXTFILE;
 
-CREATE TABLE DECIMAL_4_2(key decimal, value decimal) 
+CREATE TABLE DECIMAL_4_2(key decimal(35,25), value decimal(35,25)) 
 STORED AS ORC;
 
-LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_4_1;
+LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_4_1;
 
 INSERT OVERWRITE TABLE DECIMAL_4_2 SELECT key, key * 3 FROM DECIMAL_4_1;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_5.q
new file mode 100644
index 0000000000000000000000000000000000000000..70e5db0f701826dcb34f5643171d1c133843b8d0
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_5.q
@@ -0,0 +1,18 @@
+DROP TABLE IF EXISTS DECIMAL_5;
+
+CREATE TABLE DECIMAL_5(key decimal(10,5), value int)
+ROW FORMAT DELIMITED
+   FIELDS TERMINATED BY ' '
+STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_5;
+
+SELECT key FROM DECIMAL_5 ORDER BY key;
+
+SELECT DISTINCT key FROM DECIMAL_5 ORDER BY key;
+
+SELECT cast(key as decimal) FROM DECIMAL_5;
+
+SELECT cast(key as decimal(6,3)) FROM DECIMAL_5;
+
+DROP TABLE DECIMAL_5;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_6.q
new file mode 100644
index 0000000000000000000000000000000000000000..b58e22425612264f44266213503fd06566cd7007
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_6.q
@@ -0,0 +1,27 @@
+DROP TABLE IF EXISTS DECIMAL_6_1;
+DROP TABLE IF EXISTS DECIMAL_6_2;
+DROP TABLE IF EXISTS DECIMAL_6_3;
+
+CREATE TABLE DECIMAL_6_1(key decimal(10,5), value int)
+ROW FORMAT DELIMITED
+   FIELDS TERMINATED BY ' '
+STORED AS TEXTFILE;
+
+CREATE TABLE DECIMAL_6_2(key decimal(17,4), value int)
+ROW FORMAT DELIMITED
+   FIELDS TERMINATED BY ' '
+STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../../data/files/kv9.txt' INTO TABLE DECIMAL_6_1;
+LOAD DATA LOCAL INPATH '../../data/files/kv9.txt' INTO TABLE DECIMAL_6_2;
+
+SELECT T.key from (
+  SELECT key, value from DECIMAL_6_1
+  UNION ALL
+  SELECT key, value from DECIMAL_6_2
+) T order by T.key;
+
+CREATE TABLE DECIMAL_6_3 AS SELECT key + 5.5 AS k, value * 11 AS v from DECIMAL_6_1 ORDER BY v;
+
+desc DECIMAL_6_3;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q
index 589fc6597dfa43d7f24d8c5fef47cf35a4800d5d..86c14d9351ac1cf9a66b34a739a3b56405f8d614 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q
@@ -1,6 +1,6 @@
 -- HIVE-5292 Join on decimal columns fails
 
-create table src_dec (key decimal, value string);
-load data local inpath '../data/files/kv1.txt' into table src_dec;
+create table src_dec (key decimal(3,0), value string);
+load data local inpath '../../data/files/kv1.txt' into table src_dec;
 
 select * from src_dec a join src_dec b on a.key=b.key+450;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q
index 403c2be3fbc10dc6e18444055cea47d061ed4322..739352f9ef1e5f624c3b80fa8c25111a0a69e8e4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q
@@ -1,11 +1,11 @@
 DROP TABLE IF EXISTS DECIMAL_PRECISION;
 
-CREATE TABLE DECIMAL_PRECISION(dec decimal) 
+CREATE TABLE DECIMAL_PRECISION(dec decimal(20,10))
 ROW FORMAT DELIMITED
    FIELDS TERMINATED BY ' '
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/kv8.txt' INTO TABLE DECIMAL_PRECISION;
+LOAD DATA LOCAL INPATH '../../data/files/kv8.txt' INTO TABLE DECIMAL_PRECISION;
 
 SELECT * FROM DECIMAL_PRECISION ORDER BY dec;
 
@@ -15,13 +15,14 @@ SELECT dec, dec / 9 FROM DECIMAL_PRECISION ORDER BY dec;
 SELECT dec, dec / 27 FROM DECIMAL_PRECISION ORDER BY dec;
 SELECT dec, dec * dec FROM DECIMAL_PRECISION ORDER BY dec;
 
+EXPLAIN SELECT avg(dec), sum(dec) FROM DECIMAL_PRECISION;
 SELECT avg(dec), sum(dec) FROM DECIMAL_PRECISION;
 
-SELECT dec * cast('123456789012345678901234567890.123456789' as decimal) FROM DECIMAL_PRECISION LIMIT 1;
-SELECT * from DECIMAL_PRECISION WHERE dec > cast('123456789012345678901234567890.123456789' as decimal) LIMIT 1;
-SELECT dec * 123456789012345678901234567890.123456789 FROM DECIMAL_PRECISION LIMIT 1;
+SELECT dec * cast('12345678901234567890.12345678' as decimal(38,18)) FROM DECIMAL_PRECISION LIMIT 1;
+SELECT * from DECIMAL_PRECISION WHERE dec > cast('1234567890123456789012345678.12345678' as decimal(38,18)) LIMIT 1;
+SELECT dec * 12345678901234567890.12345678 FROM DECIMAL_PRECISION LIMIT 1;
 
-SELECT MIN(cast('123456789012345678901234567890.123456789' as decimal)) FROM DECIMAL_PRECISION;
-SELECT COUNT(cast('123456789012345678901234567890.123456789' as decimal)) FROM DECIMAL_PRECISION;
+SELECT MIN(cast('12345678901234567890.12345678' as decimal(38,18))) FROM DECIMAL_PRECISION;
+SELECT COUNT(cast('12345678901234567890.12345678' as decimal(38,18))) FROM DECIMAL_PRECISION;
 
 DROP TABLE DECIMAL_PRECISION;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q
index 3556807705ae55c2785440e08c594cb9f02ed590..cf3a86cd4d78f3b5f487a7c9924302a6b25eedbd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q
@@ -8,7 +8,7 @@ ROW FORMAT DELIMITED
    FIELDS TERMINATED BY ' '
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_TEXT;
+LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_TEXT;
 
 SELECT * FROM DECIMAL_TEXT ORDER BY key, value;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q
index b5ff088d1613a92b41948e7d48b0d042fdd52a37..0c9f1b86a9e973c1b6789a3c05c5fec8376c0295 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q
@@ -1,11 +1,13 @@
+set hive.fetch.task.conversion=more;
+
 DROP TABLE IF EXISTS DECIMAL_UDF;
 
-CREATE TABLE DECIMAL_UDF (key decimal, value int) 
+CREATE TABLE DECIMAL_UDF (key decimal(20,10), value int)
 ROW FORMAT DELIMITED
    FIELDS TERMINATED BY ' '
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_UDF;
+LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_UDF;
 
 -- addition
 EXPLAIN SELECT key + key FROM DECIMAL_UDF;
@@ -70,8 +72,8 @@ EXPLAIN SELECT abs(key) FROM DECIMAL_UDF;
 SELECT abs(key) FROM DECIMAL_UDF;
 
 -- avg
-EXPLAIN SELECT value, sum(key) / count(key), avg(key) FROM DECIMAL_UDF GROUP BY value ORDER BY value;
-SELECT value, sum(key) / count(key), avg(key) FROM DECIMAL_UDF GROUP BY value ORDER BY value;
+EXPLAIN SELECT value, sum(key) / count(key), avg(key), sum(key) FROM DECIMAL_UDF GROUP BY value ORDER BY value;
+SELECT value, sum(key) / count(key), avg(key), sum(key) FROM DECIMAL_UDF GROUP BY value ORDER BY value;
 
 -- negative
 EXPLAIN SELECT -key FROM DECIMAL_UDF;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q
index 112ac57c3be5b23c839d2c4e2dced0fb6b5296fe..14d508c07dd3d6d8fc076a87b65af94bede925cb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q
@@ -3,7 +3,7 @@ row format delimited
 fields terminated by '\t'
 lines terminated by '\n'
 stored as textfile;
-LOAD DATA LOCAL INPATH '../data/files/in7.txt' INTO TABLE impressions;
+LOAD DATA LOCAL INPATH '../../data/files/in7.txt' INTO TABLE impressions;
 
 select * from impressions;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/desc_tbl_part_cols.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/desc_tbl_part_cols.q
new file mode 100644
index 0000000000000000000000000000000000000000..89e49311fa48d64e93f87c0e684e7cd035ebf10a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/desc_tbl_part_cols.q
@@ -0,0 +1,7 @@
+create table t1 (a int, b string) partitioned by (c int, d string);
+describe t1;
+
+set hive.display.partition.cols.separately=false;
+describe t1;
+
+set hive.display.partition.cols.separately=true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q
index 6ea4156b3493efc233ca2087d7f5cd566e0b6811..81a5b3a6e6c820dc9d6be26c9f1144bb80e11eff 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q
@@ -1,9 +1,9 @@
 set hive.fileformat.check = false;
 create table kv_fileformat_check_txt (key string, value string) stored as textfile;
-load data local inpath '../data/files/kv1.seq' overwrite into table kv_fileformat_check_txt;
+load data local inpath '../../data/files/kv1.seq' overwrite into table kv_fileformat_check_txt;
 
 create table kv_fileformat_check_seq (key string, value string) stored as sequencefile;
-load data local inpath '../data/files/kv1.txt' overwrite into table kv_fileformat_check_seq;
+load data local inpath '../../data/files/kv1.txt' overwrite into table kv_fileformat_check_seq;
 
 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q
index 2a1e7276cacfe940c913cee604fd41527115f1a6..991b930d54ca4ddf418bcdfaf9efa544f546da41 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q
@@ -1,7 +1,9 @@
+set hive.fetch.task.conversion=more;
+
 SET hive.metastore.disallow.incompatible.col.type.changes=false;
 SELECT * FROM src LIMIT 1;
 CREATE TABLE test_table123 (a INT, b MAP<STRING, STRING>) PARTITIONED BY (ds STRING) STORED AS SEQUENCEFILE;
-INSERT OVERWRITE TABLE test_table123 PARTITION(ds="foo1") SELECT 1, MAP("a1", "b1") FROM src LIMIT 1;
+INSERT OVERWRITE TABLE test_table123 PARTITION(ds="foo1") SELECT 1, MAP("a1", "b1") FROM src tablesample (1 rows);
 SELECT * from test_table123 WHERE ds="foo1";
 -- This should now work as hive.metastore.disallow.incompatible.col.type.changes is false
 ALTER TABLE test_table123 REPLACE COLUMNS (a INT, b STRING);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/distinct_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/distinct_stats.q
new file mode 100644
index 0000000000000000000000000000000000000000..725183380b9ad5a257e605448bc285dd869e0994
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/distinct_stats.q
@@ -0,0 +1,20 @@
+set hive.stats.autogather=true;
+
+set hive.compute.query.using.stats=true;
+create table t1 (a string, b string);
+
+insert into table t1 select * from src;
+
+analyze table t1 compute statistics for columns a,b;
+
+explain 
+select count(distinct b) from t1 group by a;
+
+explain 
+select distinct(b) from t1;
+
+explain 
+select a, count(*) from t1 group by a;
+
+drop table t1;
+set hive.compute.query.using.stats = false;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q
index 798aa6d51a18588a9d55ed1c8a097f1d80e77b84..54e6a35b5adfec815989df044b5face8454409f1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q
@@ -6,7 +6,7 @@ alter table ptestfilter add partition (c=1, d=2);
 alter table ptestFilter add partition (c=2, d=1);
 alter table ptestfilter add partition (c=2, d=2);
 alter table ptestfilter add partition (c=3, d=1);
-alter table ptestfilter add partition (c=3, d=2);
+alter table ptestfilter add partition (c=30, d=2);
 show partitions ptestfilter;
 
 alter table ptestfilter drop partition (c=1, d=1);
@@ -15,6 +15,9 @@ show partitions ptestfilter;
 alter table ptestfilter drop partition (c=2);
 show partitions ptestfilter;
 
+alter table ptestfilter drop partition (c<4);
+show partitions ptestfilter;
+
 drop table ptestfilter;
 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_with_concurrency.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_with_concurrency.q
new file mode 100644
index 0000000000000000000000000000000000000000..797a27c23b01e005bad80503a83c38b39819d643
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_with_concurrency.q
@@ -0,0 +1,8 @@
+set hive.lock.numretries=1;
+set hive.lock.sleep.between.retries=1;
+set hive.support.concurrency=true;
+set hive.lock.manager=org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager;
+
+drop table if exists drop_with_concurrency_1;
+create table drop_with_concurrency_1 (c1 int);
+drop table drop_with_concurrency_1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q
index 397a2200a8972d00eaaeff6a13d0c5c9697cc862..699e58effcac07c997aaed329a9a1dd4c36b2dbe 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q
@@ -1,19 +1,19 @@
-create table dynamic_part_table(intcol int) partitioned by (partcol1 int, partcol2 int);
+create table dynamic_part_table(intcol string) partitioned by (partcol1 string, partcol2 string);
 
 set hive.exec.dynamic.partition.mode=nonstrict;
 
-insert into table dynamic_part_table partition(partcol1, partcol2) select 1, 1, 1 from src where key=150;
+insert into table dynamic_part_table partition(partcol1, partcol2) select '1', '1', '1' from src where key=150;
 
-insert into table dynamic_part_table partition(partcol1, partcol2) select 1, NULL, 1 from src where key=150;
+insert into table dynamic_part_table partition(partcol1, partcol2) select '1', NULL, '1' from src where key=150;
 
-insert into table dynamic_part_table partition(partcol1, partcol2) select 1, 1, NULL from src where key=150;
+insert into table dynamic_part_table partition(partcol1, partcol2) select '1', '1', NULL from src where key=150;
 
-insert into table dynamic_part_table partition(partcol1, partcol2) select 1, NULL, NULL from src where key=150;
+insert into table dynamic_part_table partition(partcol1, partcol2) select '1', NULL, NULL from src where key=150;
 
-explain extended select intcol from dynamic_part_table where partcol1=1 and partcol2=1;
+explain extended select intcol from dynamic_part_table where partcol1='1' and partcol2='1';
 
 set hive.exec.dynamic.partition.mode=strict;
 
-explain extended select intcol from dynamic_part_table where partcol1=1 and partcol2=1;
+explain extended select intcol from dynamic_part_table where partcol1='1' and partcol2='1';
 
-explain extended select intcol from dynamic_part_table where (partcol1=1 and partcol2=1)or (partcol1=1 and partcol2='__HIVE_DEFAULT_PARTITION__');
+explain extended select intcol from dynamic_part_table where (partcol1='1' and partcol2='1')or (partcol1='1' and partcol2='__HIVE_DEFAULT_PARTITION__');
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q
new file mode 100644
index 0000000000000000000000000000000000000000..5f1a5ce809e0cc8110da5b75f8c4bb670e605e11
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q
@@ -0,0 +1,161 @@
+set hive.optimize.sort.dynamic.partition=true;
+set hive.exec.dynamic.partition=true;
+set hive.exec.max.dynamic.partitions=1000;
+set hive.exec.max.dynamic.partitions.pernode=1000;
+set hive.exec.dynamic.partition.mode=nonstrict;
+set hive.vectorized.execution.enabled=true;
+set hive.enforce.bucketing=false;
+set hive.enforce.sorting=false;
+
+create table over1k(
+           t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+       row format delimited
+       fields terminated by '|';
+
+load data local inpath '../../data/files/over1k' into table over1k;
+
+create table over1k_orc like over1k;
+alter table over1k_orc set fileformat orc;
+insert overwrite table over1k_orc select * from over1k;
+
+create table over1k_part_orc(
+           si smallint,
+           i int,
+           b bigint,
+           f float)
+       partitioned by (ds string, t tinyint) stored as orc;
+
+create table over1k_part_limit_orc like over1k_part_orc;
+alter table over1k_part_limit_orc set fileformat orc;
+
+create table over1k_part_buck_orc(
+           si smallint,
+           i int,
+           b bigint,
+           f float)
+       partitioned by (t tinyint)
+       clustered by (si) into 4 buckets stored as orc;
+
+create table over1k_part_buck_sort_orc(
+           si smallint,
+           i int,
+           b bigint,
+           f float)
+       partitioned by (t tinyint)
+       clustered by (si) 
+       sorted by (f) into 4 buckets stored as orc;
+
+-- map-only jobs converted to map-reduce job by hive.optimize.sort.dynamic.partition optimization
+explain insert overwrite table over1k_part_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by si;
+explain insert overwrite table over1k_part_limit_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 limit 10;
+explain insert overwrite table over1k_part_buck_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27;
+explain insert overwrite table over1k_part_buck_sort_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27;
+
+insert overwrite table over1k_part_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by si;
+insert overwrite table over1k_part_limit_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 limit 10;
+insert overwrite table over1k_part_buck_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27;
+insert overwrite table over1k_part_buck_sort_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27;
+
+set hive.enforce.bucketing=true;
+set hive.enforce.sorting=true;
+
+-- map-reduce jobs modified by hive.optimize.sort.dynamic.partition optimization
+explain insert into table over1k_part_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by si;
+explain insert into table over1k_part_limit_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 limit 10;
+explain insert into table over1k_part_buck_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27;
+explain insert into table over1k_part_buck_sort_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27;
+
+insert into table over1k_part_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by si;
+insert into table over1k_part_limit_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 limit 10;
+insert into table over1k_part_buck_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27;
+insert into table over1k_part_buck_sort_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27;
+
+desc formatted over1k_part_orc partition(ds="foo",t=27);
+desc formatted over1k_part_orc partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__");
+desc formatted over1k_part_limit_orc partition(ds="foo",t=27);
+desc formatted over1k_part_limit_orc partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__");
+desc formatted over1k_part_buck_orc partition(t=27);
+desc formatted over1k_part_buck_orc partition(t="__HIVE_DEFAULT_PARTITION__");
+desc formatted over1k_part_buck_sort_orc partition(t=27);
+desc formatted over1k_part_buck_sort_orc partition(t="__HIVE_DEFAULT_PARTITION__");
+
+select count(*) from over1k_part_orc;
+select count(*) from over1k_part_limit_orc;
+select count(*) from over1k_part_buck_orc;
+select count(*) from over1k_part_buck_sort_orc;
+
+-- tests for HIVE-6883
+create table over1k_part2_orc(
+           si smallint,
+           i int,
+           b bigint,
+           f float)
+       partitioned by (ds string, t tinyint);
+
+set hive.optimize.sort.dynamic.partition=false;
+explain insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by i;
+set hive.optimize.sort.dynamic.partition=true;
+explain insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by i;
+
+set hive.optimize.sort.dynamic.partition=false;
+insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by i;
+
+desc formatted over1k_part2_orc partition(ds="foo",t=27);
+desc formatted over1k_part2_orc partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__");
+
+select * from over1k_part2_orc;
+select count(*) from over1k_part2_orc;
+
+set hive.optimize.sort.dynamic.partition=true;
+insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by i;
+
+desc formatted over1k_part2_orc partition(ds="foo",t=27);
+desc formatted over1k_part2_orc partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__");
+
+select * from over1k_part2_orc;
+select count(*) from over1k_part2_orc;
+
+-- hadoop-1 does not honor number of reducers in local mode. There is always only 1 reducer irrespective of the number of buckets.
+-- Hence all records go to one bucket and all other buckets will be empty. Similar to HIVE-6867. However, hadoop-2 honors number
+-- of reducers and records are spread across all reducers. To avoid this inconsistency we will make number of buckets to 1 for this test.
+create table over1k_part_buck_sort2_orc(
+           si smallint,
+           i int,
+           b bigint,
+           f float)
+       partitioned by (t tinyint)
+       clustered by (si)
+       sorted by (f) into 1 buckets;
+
+set hive.optimize.sort.dynamic.partition=false;
+explain insert overwrite table over1k_part_buck_sort2_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27;
+set hive.optimize.sort.dynamic.partition=true;
+explain insert overwrite table over1k_part_buck_sort2_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27;
+
+set hive.optimize.sort.dynamic.partition=false;
+insert overwrite table over1k_part_buck_sort2_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27;
+
+desc formatted over1k_part_buck_sort2_orc partition(t=27);
+desc formatted over1k_part_buck_sort2_orc partition(t="__HIVE_DEFAULT_PARTITION__");
+
+select * from over1k_part_buck_sort2_orc;
+select count(*) from over1k_part_buck_sort2_orc;
+
+set hive.optimize.sort.dynamic.partition=true;
+insert overwrite table over1k_part_buck_sort2_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27;
+
+desc formatted over1k_part_buck_sort2_orc partition(t=27);
+desc formatted over1k_part_buck_sort2_orc partition(t="__HIVE_DEFAULT_PARTITION__");
+
+select * from over1k_part_buck_sort2_orc;
+select count(*) from over1k_part_buck_sort2_orc;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q
new file mode 100644
index 0000000000000000000000000000000000000000..52b5d1e0c1d8ad105810d178071e9ffe9df393af
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q
@@ -0,0 +1,155 @@
+set hive.optimize.sort.dynamic.partition=true;
+set hive.exec.dynamic.partition=true;
+set hive.exec.max.dynamic.partitions=1000;
+set hive.exec.max.dynamic.partitions.pernode=1000;
+set hive.exec.dynamic.partition.mode=nonstrict;
+set hive.enforce.bucketing=false;
+set hive.enforce.sorting=false;
+
+create table over1k(
+           t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+       row format delimited
+       fields terminated by '|';
+
+load data local inpath '../../data/files/over1k' into table over1k;
+
+create table over1k_part(
+           si smallint,
+           i int,
+           b bigint,
+           f float)
+       partitioned by (ds string, t tinyint);
+
+create table over1k_part_limit like over1k_part;
+
+create table over1k_part_buck(
+           si smallint,
+           i int,
+           b bigint,
+           f float)
+       partitioned by (t tinyint)
+       clustered by (si) into 4 buckets;
+
+create table over1k_part_buck_sort(
+           si smallint,
+           i int,
+           b bigint,
+           f float)
+       partitioned by (t tinyint)
+       clustered by (si) 
+       sorted by (f) into 4 buckets;
+
+-- map-only jobs converted to map-reduce job by hive.optimize.sort.dynamic.partition optimization
+explain insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27;
+explain insert overwrite table over1k_part_limit partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27 limit 10;
+explain insert overwrite table over1k_part_buck partition(t) select si,i,b,f,t from over1k where t is null or t=27;
+explain insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,t from over1k where t is null or t=27;
+
+insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27;
+insert overwrite table over1k_part_limit partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27 limit 10;
+insert overwrite table over1k_part_buck partition(t) select si,i,b,f,t from over1k where t is null or t=27;
+insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,t from over1k where t is null or t=27;
+
+set hive.enforce.bucketing=true;
+set hive.enforce.sorting=true;
+
+-- map-reduce jobs modified by hive.optimize.sort.dynamic.partition optimization
+explain insert into table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27;
+explain insert into table over1k_part_limit partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27 limit 10;
+explain insert into table over1k_part_buck partition(t) select si,i,b,f,t from over1k where t is null or t=27;
+explain insert into table over1k_part_buck_sort partition(t) select si,i,b,f,t from over1k where t is null or t=27;
+
+insert into table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27;
+insert into table over1k_part_limit partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27 limit 10;
+insert into table over1k_part_buck partition(t) select si,i,b,f,t from over1k where t is null or t=27;
+insert into table over1k_part_buck_sort partition(t) select si,i,b,f,t from over1k where t is null or t=27;
+
+desc formatted over1k_part partition(ds="foo",t=27);
+desc formatted over1k_part partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__");
+desc formatted over1k_part_limit partition(ds="foo",t=27);
+desc formatted over1k_part_limit partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__");
+desc formatted over1k_part_buck partition(t=27);
+desc formatted over1k_part_buck partition(t="__HIVE_DEFAULT_PARTITION__");
+desc formatted over1k_part_buck_sort partition(t=27);
+desc formatted over1k_part_buck_sort partition(t="__HIVE_DEFAULT_PARTITION__");
+
+select count(*) from over1k_part;
+select count(*) from over1k_part_limit;
+select count(*) from over1k_part_buck;
+select count(*) from over1k_part_buck_sort;
+
+-- tests for HIVE-6883
+create table over1k_part2(
+           si smallint,
+           i int,
+           b bigint,
+           f float)
+       partitioned by (ds string, t tinyint);
+
+set hive.optimize.sort.dynamic.partition=false;
+explain insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from over1k where t is null or t=27 order by i;
+set hive.optimize.sort.dynamic.partition=true;
+explain insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from over1k where t is null or t=27 order by i;
+
+set hive.optimize.sort.dynamic.partition=false;
+insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from over1k where t is null or t=27 order by i;
+
+desc formatted over1k_part2 partition(ds="foo",t=27);
+desc formatted over1k_part2 partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__");
+
+select * from over1k_part2;
+select count(*) from over1k_part2;
+
+set hive.optimize.sort.dynamic.partition=true;
+insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from over1k where t is null or t=27 order by i;
+
+desc formatted over1k_part2 partition(ds="foo",t=27);
+desc formatted over1k_part2 partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__");
+
+select * from over1k_part2;
+select count(*) from over1k_part2;
+
+-- hadoop-1 does not honor number of reducers in local mode. There is always only 1 reducer irrespective of the number of buckets.
+-- Hence all records go to one bucket and all other buckets will be empty. Similar to HIVE-6867. However, hadoop-2 honors number
+-- of reducers and records are spread across all reducers. To avoid this inconsistency we will make number of buckets to 1 for this test.
+create table over1k_part_buck_sort2(
+           si smallint,
+           i int,
+           b bigint,
+           f float)
+       partitioned by (t tinyint)
+       clustered by (si)
+       sorted by (f) into 1 buckets;
+
+set hive.optimize.sort.dynamic.partition=false;
+explain insert overwrite table over1k_part_buck_sort2 partition(t) select si,i,b,f,t from over1k where t is null or t=27;
+set hive.optimize.sort.dynamic.partition=true;
+explain insert overwrite table over1k_part_buck_sort2 partition(t) select si,i,b,f,t from over1k where t is null or t=27;
+
+set hive.optimize.sort.dynamic.partition=false;
+insert overwrite table over1k_part_buck_sort2 partition(t) select si,i,b,f,t from over1k where t is null or t=27;
+
+desc formatted over1k_part_buck_sort2 partition(t=27);
+desc formatted over1k_part_buck_sort2 partition(t="__HIVE_DEFAULT_PARTITION__");
+
+select * from over1k_part_buck_sort2;
+select count(*) from over1k_part_buck_sort2;
+
+set hive.optimize.sort.dynamic.partition=true;
+insert overwrite table over1k_part_buck_sort2 partition(t) select si,i,b,f,t from over1k where t is null or t=27;
+
+desc formatted over1k_part_buck_sort2 partition(t=27);
+desc formatted over1k_part_buck_sort2 partition(t="__HIVE_DEFAULT_PARTITION__");
+
+select * from over1k_part_buck_sort2;
+select count(*) from over1k_part_buck_sort2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q
index a7f4cf79c980f4a6d5a6a03e900602c6ecc361a1..d29a7a8f105d6a8927acb918ed37f499a7e47e33 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q
@@ -8,7 +8,7 @@ DROP TABLE escape1;
 DROP TABLE escape_raw;
 
 CREATE TABLE escape_raw (s STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/escapetest.txt' INTO TABLE  escape_raw;
+LOAD DATA LOCAL INPATH '../../data/files/escapetest.txt' INTO TABLE  escape_raw;
 
 SELECT count(*) from escape_raw;
 SELECT * from escape_raw;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q
index 473cbf8c94e4edf1bd094d0173856d8b48881ecb..24601343b1474253c01e11501c5467900cd78863 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q
@@ -10,7 +10,7 @@ DROP TABLE IF EXISTS escape2;
 DROP TABLE IF EXISTS escape_raw;
 
 CREATE TABLE escape_raw (s STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/escapetest.txt' INTO TABLE  escape_raw;
+LOAD DATA LOCAL INPATH '../../data/files/escapetest.txt' INTO TABLE  escape_raw;
 
 SELECT count(*) from escape_raw;
 SELECT * from escape_raw;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q
index 6e8bf8ebc1fa29f4376ed83dcbba7d4b321982bd..4be6e3f6d876bbc3855051c06ff869c062e2a2d4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q
@@ -3,7 +3,7 @@ CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING);
 SHOW PARTITIONS exchange_part_test1;
 SHOW PARTITIONS exchange_part_test2;
 
-ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05');
+ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05');
 SHOW PARTITIONS exchange_part_test1;
 SHOW PARTITIONS exchange_part_test2;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q
index 27b335a3d7844b38a664f09395e9ff211db2d6da..f346ddeec496385ce907eac2713e818fbbe09f95 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q
@@ -3,7 +3,7 @@ CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRIN
 SHOW PARTITIONS exchange_part_test1;
 SHOW PARTITIONS exchange_part_test2;
 
-ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='1');
+ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='1');
 SHOW PARTITIONS exchange_part_test1;
 SHOW PARTITIONS exchange_part_test2;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q
index 7b9060d420a1e67099394219be56a85474218ba9..7c076cebe8ad4a878a1d868f9b2f3ed0b3e2c3b0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q
@@ -3,8 +3,9 @@ CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRIN
 SHOW PARTITIONS exchange_part_test1;
 SHOW PARTITIONS exchange_part_test2;
 
-ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='1');
-ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='2');
+ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2014-01-03', hr='1');
+ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='1');
+ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='2');
 SHOW PARTITIONS exchange_part_test1;
 SHOW PARTITIONS exchange_part_test2;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q
index 7fa96b629a0faa283356331f0572092664fcb76a..8288bbfd86accaae3937ed40ded720f0be52fc37 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q
@@ -1,3 +1,6 @@
+set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest;
+set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator;
+
 set hive.test.mode=true;
 set hive.test.mode.prefix=;
 set hive.test.mode.nosamplelist=exim_department,exim_employee;
@@ -5,8 +8,8 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -16,7 +19,7 @@ use importer;
 import from 'ql/test/data/exports/exim_department';
 describe extended exim_department;
 show table extended like exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 select * from exim_department;
 drop table exim_department;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q
index 9920e778d18a6ba73810a6965f96735536e70c5c..1e2eed803a01da1bad05f7978438543f57dd1e6b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q
@@ -5,9 +5,9 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;		
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -17,7 +17,7 @@ use importer;
 import from 'ql/test/data/exports/exim_department';
 describe extended exim_department;
 show table extended like exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 select * from exim_department;
 drop table exim_department;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q
index 4017c83aa3dd4c3b070060ded88bd0aa5d6961c9..474a5a4a5bb5fe7bdb7b5b4a230faccc0a2cada2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q
@@ -7,8 +7,8 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -18,7 +18,7 @@ use importer;
 import from 'ql/test/data/exports/exim_employee';
 describe extended exim_employee;
 show table extended like exim_employee;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 select * from exim_employee;
 drop table exim_employee;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q
index 21138f0263418e72493e4b42975770a1a51f8d2e..dbd2c6bf5c4c5a434859725a6d8e9897e0e5ec5f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q
@@ -7,10 +7,10 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -20,7 +20,7 @@ use importer;
 import from 'ql/test/data/exports/exim_employee';
 describe extended exim_employee;
 show table extended like exim_employee;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 select * from exim_employee;
 drop table exim_employee;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q
index 5f6bdee83cc908344380cb4f31c8fd5d9380c642..47d949aa36d9b7a08f4dfed819da1bce5c11defc 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q
@@ -5,9 +5,9 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -21,6 +21,6 @@ import from 'ql/test/data/exports/exim_department';
 describe extended exim_department;
 select * from exim_department;
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_all_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_all_part.q
index 69c6faa30a07cf90789c6d4a30ec03346a87b014..b2567fb2703263267904ef6c3657611c0a2058e3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_all_part.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_all_part.q
@@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -26,7 +26,7 @@ use importer;
 import from 'ql/test/data/exports/exim_employee';
 describe extended exim_employee;
 show table extended like exim_employee;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 select * from exim_employee;
 drop table exim_employee;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q
index cdc02fa25c169d53a3cd91c1a72fca90737ac12d..82df69874b47ac06b30da5c022fe6de79aa26ee1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q
@@ -19,8 +19,8 @@ alter table exim_employee set fileformat
 	outputformat "org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat";
 
 alter table exim_employee add partition (emp_country='in', emp_state='ka');
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -32,7 +32,7 @@ describe extended exim_employee;
 describe extended exim_employee partition (emp_country='in', emp_state='tn');
 describe extended exim_employee partition (emp_country='in', emp_state='ka');
 show table extended like exim_employee;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 select * from exim_employee;
 drop table exim_employee;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q
index 50a59463b1870895e18c09cd3090008b2e2361fa..a2c977356c84cd7e23203a71e4c75135212ea951 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q
@@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee partition (emp_state="ka") to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -26,7 +26,7 @@ use importer;
 import from 'ql/test/data/exports/exim_employee';
 describe extended exim_employee;
 show table extended like exim_employee;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 select * from exim_employee;
 drop table exim_employee;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q
index 5136090929fc5bfe0cce383f45154f75dbba219c..3a612964222ebc2df4473985f6cf05552032fdd1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q
@@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee partition (emp_country="in",emp_state="ka") to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -26,7 +26,7 @@ use importer;
 import from 'ql/test/data/exports/exim_employee';
 describe extended exim_employee;
 show table extended like exim_employee;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 select * from exim_employee;
 drop table exim_employee;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q
index 5b9d4ddc03c59f47dd8a0d3f52efca0a71ef4f8d..8c774d5a8ba846d724ead55bc0e8c01b2b5d5354 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q
@@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -28,12 +28,12 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "iso code", emp_state string comment "free-form text")
 	stored as textfile	
 	tblproperties("maker"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="al");			
 import from 'ql/test/data/exports/exim_employee';
 describe extended exim_employee;
 select * from exim_employee;
 drop table exim_employee;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q
index 173f1569c501a42369d9726622354b960aeb9448..8a1d9454763c7139468a316b83a2a6a63d4b9a28 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q
@@ -5,9 +5,9 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee,exim_imported_dept
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;		
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -17,12 +17,12 @@ create table exim_department ( dep_id int comment "department id")
 	partitioned by (emp_org string)
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department partition (emp_org="hr");
+load data local inpath "../../data/files/test.dat" into table exim_department partition (emp_org="hr");
 import table exim_imported_dept from 'ql/test/data/exports/exim_department';
 describe extended exim_imported_dept;
 select * from exim_imported_dept;
 drop table exim_imported_dept;
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q
index 178b76674d862508d327064c5c2d9c2369d04240..53fc2936bb2f3b34e0c5885849f52f3b956164b8 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q
@@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -27,14 +27,14 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
 import table exim_employee partition (emp_country="us", emp_state="tn") from 'ql/test/data/exports/exim_employee';
 describe extended exim_employee;
 select * from exim_employee;
 drop table exim_employee;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_10_external_managed.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_10_external_managed.q
index 413f2aa4762f5d551e754b68c15f0dde8bbb2f45..54859eed196cd73f2399ca5829cfea744d53d8ab 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_10_external_managed.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_10_external_managed.q
@@ -2,18 +2,18 @@ set hive.test.mode=true;
 set hive.test.mode.prefix=;
 set hive.test.mode.nosamplelist=exim_department,exim_employee;
 
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_department/temp;
-dfs -rmr ../build/ql/test/data/tablestore/exim_department;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_department;
 create external table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	location 'ql/test/data/tablestore/exim_department'
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;		
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/tablestore/exim_department;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_department;
 
 create database importer;
 use importer;
@@ -22,6 +22,6 @@ import from 'ql/test/data/exports/exim_department';
 describe extended exim_department;
 select * from exim_department;
 drop table exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_11_managed_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_11_managed_external.q
index f3b2896a322a5893afc70f40260382d9166ae956..4fc39dcff0620c8d11f1ef834f2f29770d4a9713 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_11_managed_external.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_11_managed_external.q
@@ -5,9 +5,9 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;		
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -17,7 +17,7 @@ use importer;
 import external table exim_department from 'ql/test/data/exports/exim_department';
 describe extended exim_department;
 select * from exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 select * from exim_department;
 drop table exim_department;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q
index 37d063432e16a56048c7e312af44ebcb64d9fa03..e4d50ffe5b8f088e0327ef9f87bb408fb2299bd0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q
@@ -5,24 +5,24 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;		
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/ql/test/data/exports/exim_department/temp;
+dfs -rmr ${system:test.tmp.dir}/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
 create database importer;
 use importer;
 
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_department/temp;
-dfs -rmr ../build/ql/test/data/tablestore/exim_department;
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/ql/test/data/tablestore/exim_department/temp;
+dfs -rmr ${system:test.tmp.dir}/ql/test/data/tablestore/exim_department;
 
 import external table exim_department from 'ql/test/data/exports/exim_department' 
 	location 'ql/test/data/tablestore/exim_department';
 describe extended exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr ${system:test.tmp.dir}/ql/test/data/exports/exim_department;
 select * from exim_department;
-dfs -rmr ../build/ql/test/data/tablestore/exim_department;
+dfs -rmr ${system:test.tmp.dir}/ql/test/data/tablestore/exim_department;
 select * from exim_department;
 drop table exim_department;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q
index fb5058b840bae609ca226b0a5d20056ea54c30d9..909d23794be9d77fcbcb208d300bafaf4a6a2b37 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q
@@ -5,24 +5,24 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;		
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
 create database importer;
 use importer;
 
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_department/temp;
-dfs -rmr ../build/ql/test/data/tablestore/exim_department;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_department;
 
 import table exim_department from 'ql/test/data/exports/exim_department' 
 	location 'ql/test/data/tablestore/exim_department';
 describe extended exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 select * from exim_department;
-dfs -rmr ../build/ql/test/data/tablestore/exim_department;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_department;
 select * from exim_department;
 drop table exim_department;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q
index 031b6bda6cece7309d7161ae14000e1ab02e60a7..dbb5fd93435a08953cad4bf60c4d24c89a8f8e08 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q
@@ -5,17 +5,17 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee;
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" into table exim_department;		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;		
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
 create database importer;
 use importer;
 
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_department/temp;
-dfs -rmr ../build/ql/test/data/tablestore/exim_department;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_department;
 
 create table exim_department ( dep_id int comment "department id") 	
 	stored as textfile
@@ -24,9 +24,9 @@ create table exim_department ( dep_id int comment "department id")
 import table exim_department from 'ql/test/data/exports/exim_department'
 	location 'ql/test/data/tablestore/exim_department';
 describe extended exim_department;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 select * from exim_department;
-dfs -rmr ../build/ql/test/data/tablestore/exim_department;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_department;
 select * from exim_department;	
 drop table exim_department;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q
index ff088c70d7ffb5c45ccc26712e39092e79ac866b..989dd6cf565471c1981ea08421d133d2df339c78 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q
@@ -7,24 +7,24 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
 create database importer;
 use importer;
 
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/tablestore/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee;
 
 create external table exim_employee ( emp_id int comment "employee id") 	
 	comment "employee table"
@@ -32,17 +32,17 @@ create external table exim_employee ( emp_id int comment "employee id")
 	stored as textfile	
 	location 'ql/test/data/tablestore/exim_employee'
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
 import external table exim_employee partition (emp_country="us", emp_state="tn") 
 	from 'ql/test/data/exports/exim_employee';
 describe extended exim_employee;
 select * from exim_employee;	
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 select * from exim_employee;
-dfs -rmr ../build/ql/test/data/tablestore/exim_employee;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee;
 select * from exim_employee;
 drop table exim_employee;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q
index 6f4ee7a01c25648375937879e5c309cf4e8eea16..7eec358850e294c206ab14045beb5c9cc9bb186e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q
@@ -7,26 +7,26 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
 create database importer;
 use importer;
 
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/tablestore/exim_employee;
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore2/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/tablestore2/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore2/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/tablestore2/exim_employee;
 
 create external table exim_employee ( emp_id int comment "employee id") 	
 	comment "employee table"
@@ -39,11 +39,11 @@ import table exim_employee partition (emp_country="us", emp_state="tn")
 	location 'ql/test/data/tablestore/exim_employee';
 show table extended like exim_employee;
 show table extended like exim_employee partition (emp_country="us", emp_state="tn");		
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 select * from exim_employee;
-dfs -rmr ../build/ql/test/data/tablestore/exim_employee;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee;
 select * from exim_employee;
 drop table exim_employee;
-dfs -rmr ../build/ql/test/data/tablestore2/exim_employee;
+dfs -rmr target/tmp/ql/test/data/tablestore2/exim_employee;
 
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q
index 56ec152948aedf6fa3f4a2eb0b9bcf98b57a2416..20cd7e0513c7bfd783be75d80d44ca8f9b0f42f1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q
@@ -7,24 +7,24 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
 create database importer;
 use importer;
 
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/tablestore/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee;
 
 create table exim_employee ( emp_id int comment "employee id") 	
 	comment "employee table"
@@ -39,9 +39,9 @@ alter table exim_employee add partition	(emp_country="us", emp_state="ap")
 show table extended like exim_employee;
 show table extended like exim_employee partition (emp_country="us", emp_state="tn");
 show table extended like exim_employee partition (emp_country="us", emp_state="ap");	
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 select * from exim_employee;
-dfs -rmr ../build/ql/test/data/tablestore/exim_employee;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee;
 select * from exim_employee;
 drop table exim_employee;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q
index 7aa1297dc7d027a036901b6a557526ce1fb58911..a300b1dbf128529e931c9d60088a267ffd6efbca 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q
@@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -29,7 +29,7 @@ describe extended exim_employee;
 show table extended like exim_employee;
 show table extended like exim_employee partition (emp_country="us", emp_state="tn");
 select * from exim_employee;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 select * from exim_employee;
 drop table exim_employee;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q
index cb9f8efc087dae660ab16aab68b6ad5297d518e1..a821c75d70ccaad907ae975a930f4ff183635a00 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q
@@ -7,20 +7,20 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test2.dat" 
+load data local inpath "../../data/files/test2.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
 create database importer;
 use importer;
 
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/tablestore/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee;
 
 import external table exim_employee 
 	from 'ql/test/data/exports/exim_employee'
@@ -29,9 +29,9 @@ describe extended exim_employee;
 show table extended like exim_employee;
 show table extended like exim_employee partition (emp_country="in", emp_state="tn");
 show table extended like exim_employee partition (emp_country="in", emp_state="ka");
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 select * from exim_employee;
-dfs -rmr ../build/ql/test/data/tablestore/exim_employee;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee;
 select * from exim_employee;
 drop table exim_employee;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q
index bdbd19df70a21154da6200bdc2b2d7b6506d9b3a..be1216453bc180ff543953a7ede19a3136fa975a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q
@@ -7,24 +7,24 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
 create database importer;
 use importer;
 
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/tablestore/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee;
 
 import external table exim_employee partition (emp_country="us", emp_state="tn") 
 	from 'ql/test/data/exports/exim_employee'
@@ -32,9 +32,9 @@ import external table exim_employee partition (emp_country="us", emp_state="tn")
 describe extended exim_employee;	
 show table extended like exim_employee;
 show table extended like exim_employee partition (emp_country="us", emp_state="tn");
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 select * from exim_employee;
-dfs -rmr ../build/ql/test/data/tablestore/exim_employee;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee;
 select * from exim_employee;
 drop table exim_employee;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q
index eb44961a9b7cad6a1d6225e0b16305a1dde137b6..000904aa6634fcb225f0dd1a7bd166bbefb85991 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q
@@ -7,24 +7,24 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="ka");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="tn");	
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="us", emp_state="ka");		
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
 create database importer;
 use importer;
 
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/tablestore/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee;
 
 import table exim_employee partition (emp_country="us", emp_state="tn") 
 	from 'ql/test/data/exports/exim_employee'
@@ -32,9 +32,9 @@ import table exim_employee partition (emp_country="us", emp_state="tn")
 describe extended exim_employee;	
 show table extended like exim_employee;
 show table extended like exim_employee partition (emp_country="us", emp_state="tn");
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 select * from exim_employee;
-dfs -rmr ../build/ql/test/data/tablestore/exim_employee;
+dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee;
 select * from exim_employee;
 drop table exim_employee;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q
index 822ed70a38c0c69806695b1ee55f4d8f2cafabd0..293a011cb20f697fae23a73fe2ed744aed8d4864 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q
@@ -2,13 +2,13 @@ set hive.test.mode=true;
 set hive.test.mode.prefix=;
 
 create table exim_department ( dep_id int) stored as textfile;
-load data local inpath "../data/files/test.dat" into table exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;
 
 set hive.security.authorization.enabled=true;
 
 grant Select on table exim_department to user hive_test_user;
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 
 set hive.security.authorization.enabled=false;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q
index 440d08d2dc7b50acea90bbda26cbc447f4f93a4d..03714ab17dcbf47e0c4c8c09722700eb16d54f4e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q
@@ -3,9 +3,9 @@ set hive.test.mode.prefix=;
 set hive.test.mode.nosamplelist=exim_department,exim_employee;
 
 create table exim_department ( dep_id int) stored as textfile;
-load data local inpath "../data/files/test.dat" into table exim_department;
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -22,5 +22,5 @@ set hive.security.authorization.enabled=false;
 select * from exim_department;
 drop table exim_department;
 drop database importer;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q
index 30fc343dd8f9c51d631da4525ec5313d11942d63..cb6af0efbca62e15ff6321b9f67890e44378209c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q
@@ -7,10 +7,10 @@ create table exim_employee ( emp_id int comment "employee id")
 	partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text")
 	stored as textfile	
 	tblproperties("creator"="krishna");
-load data local inpath "../data/files/test.dat" 
+load data local inpath "../../data/files/test.dat" 
 	into table exim_employee partition (emp_country="in", emp_state="tn");
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 export table exim_employee to 'ql/test/data/exports/exim_employee';
 drop table exim_employee;
 
@@ -29,6 +29,6 @@ import from 'ql/test/data/exports/exim_employee';
 
 set hive.security.authorization.enabled=false;
 select * from exim_employee;
-dfs -rmr ../build/ql/test/data/exports/exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
 drop table exim_employee;
 drop database importer;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q
index 2dc5af6ce4774d47da6da6cf19fa7cbf536c6c5b..8934c473723847df12c65c04e64584c4c7cff275 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q
@@ -3,9 +3,9 @@ set hive.test.mode.prefix=;
 set hive.test.mode.nosamplelist=exim_department,exim_employee;
 
 create table exim_department ( dep_id int) stored as textfile;
-load data local inpath "../data/files/test.dat" into table exim_department;
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/test;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+load data local inpath "../../data/files/test.dat" into table exim_department;
+dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/test;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 export table exim_department to 'ql/test/data/exports/exim_department';
 drop table exim_department;
 
@@ -20,5 +20,5 @@ set hive.security.authorization.enabled=false;
 select * from exim_department;
 drop table exim_department;
 drop database importer;
-dfs -rmr ../build/ql/test/data/exports/exim_department;
+dfs -rmr target/tmp/ql/test/data/exports/exim_department;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_hidden_files.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_hidden_files.q
new file mode 100644
index 0000000000000000000000000000000000000000..f58c9f948d62b2dab41305be068ccb518198ec93
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_hidden_files.q
@@ -0,0 +1,22 @@
+set hive.test.mode=true;
+set hive.test.mode.prefix=;
+set hive.test.mode.nosamplelist=exim_department,exim_employee;
+
+create table exim_employee ( emp_id int) partitioned by (emp_country string);
+load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in");		
+
+dfs ${system:test.dfs.mkdir} ${system:test.warehouse.dir}/exim_employee/emp_country=in/_logs;
+dfs -touchz ${system:test.warehouse.dir}/exim_employee/emp_country=in/_logs/job.xml;
+export table exim_employee to 'ql/test/data/exports/exim_employee';
+drop table exim_employee;
+
+create database importer;
+use importer;
+
+import from 'ql/test/data/exports/exim_employee';
+describe formatted exim_employee;
+select * from exim_employee;
+dfs -rmr target/tmp/ql/test/data/exports/exim_employee;
+drop table exim_employee;
+drop database importer;
+use default;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_rearrange.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_rearrange.q
new file mode 100644
index 0000000000000000000000000000000000000000..ca2da354bd367e438a1ed2b453cb41cc6f5d3876
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_rearrange.q
@@ -0,0 +1,98 @@
+-- query from auto_sortmerge_join_9.q
+
+CREATE TABLE tbl1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS;
+CREATE TABLE tbl2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS;
+
+set hive.auto.convert.join=true;
+set hive.optimize.bucketmapjoin = true;
+set hive.optimize.bucketmapjoin.sortedmerge = true;
+set hive.auto.convert.sortmerge.join=true;
+set hive.auto.convert.sortmerge.join.to.mapjoin=false;
+
+set hive.explain.dependency.append.tasktype=true;
+
+-- default behavior
+
+explain
+select src1.key, src1.cnt1, src2.cnt1 from
+(
+  select key, count(*) as cnt1 from
+  (
+    select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key
+  ) subq1 group by key
+) src1
+join
+(
+  select key, count(*) as cnt1 from
+  (
+    select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key
+  ) subq2 group by key
+) src2
+on src1.key = src2.key
+order by src1.key, src1.cnt1, src2.cnt1;
+
+set hive.stageid.rearrange=IDONLY;
+
+-- changes id only
+
+explain
+select src1.key, src1.cnt1, src2.cnt1 from
+(
+  select key, count(*) as cnt1 from
+  (
+    select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key
+  ) subq1 group by key
+) src1
+join
+(
+  select key, count(*) as cnt1 from
+  (
+    select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key
+  ) subq2 group by key
+) src2
+on src1.key = src2.key
+order by src1.key, src1.cnt1, src2.cnt1;
+
+set hive.stageid.rearrange=TRAVERSE;
+
+-- assign ids in traverse order
+
+explain
+select src1.key, src1.cnt1, src2.cnt1 from
+(
+  select key, count(*) as cnt1 from
+  (
+    select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key
+  ) subq1 group by key
+) src1
+join
+(
+  select key, count(*) as cnt1 from
+  (
+    select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key
+  ) subq2 group by key
+) src2
+on src1.key = src2.key
+order by src1.key, src1.cnt1, src2.cnt1;
+
+set hive.stageid.rearrange=EXECUTION;
+
+-- assign ids in execution order
+
+explain
+select src1.key, src1.cnt1, src2.cnt1 from
+(
+  select key, count(*) as cnt1 from
+  (
+    select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key
+  ) subq1 group by key
+) src1
+join
+(
+  select key, count(*) as cnt1 from
+  (
+    select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key
+  ) subq2 group by key
+) src2
+on src1.key = src2.key
+order by src1.key, src1.cnt1, src2.cnt1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/external_table_with_space_in_location_path.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/external_table_with_space_in_location_path.q
new file mode 100644
index 0000000000000000000000000000000000000000..ad070464f9c4e610edfb03a01c90229def2e02bf
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/external_table_with_space_in_location_path.q
@@ -0,0 +1,23 @@
+dfs ${system:test.dfs.mkdir} hdfs:///tmp/test/;
+
+dfs -copyFromLocal ../../data/files/ext_test_space hdfs:///tmp/test/ext_test_space;
+
+CREATE EXTERNAL TABLE spacetest (id int, message string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LOCATION 'hdfs:///tmp/test/ext_test_space/folder+with space';
+
+SELECT * FROM spacetest;
+
+SELECT count(*) FROM spacetest;
+
+DROP TABLE spacetest;
+
+CREATE EXTERNAL TABLE spacetestpartition (id int, message string) PARTITIONED BY (day int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t';
+
+ALTER TABLE spacetestpartition ADD PARTITION (day=10) LOCATION 'hdfs:///tmp/test/ext_test_space/folder+with space';
+
+SELECT * FROM spacetestpartition;
+
+SELECT count(*) FROM spacetestpartition;
+
+DROP TABLE spacetestpartition;
+
+dfs -rmr hdfs:///tmp/test;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/file_with_header_footer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/file_with_header_footer.q
new file mode 100644
index 0000000000000000000000000000000000000000..8b65c7896d590e9dc868c2931016cfe97d9448a0
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/file_with_header_footer.q
@@ -0,0 +1,39 @@
+dfs ${system:test.dfs.mkdir} hdfs:///tmp/test/;
+
+dfs -copyFromLocal ../../data/files/header_footer_table_1 hdfs:///tmp/test/header_footer_table_1;
+
+dfs -copyFromLocal ../../data/files/header_footer_table_2 hdfs:///tmp/test/header_footer_table_2;
+
+dfs -copyFromLocal ../../data/files/header_footer_table_3 hdfs:///tmp/test/header_footer_table_3;
+
+CREATE EXTERNAL TABLE header_footer_table_1 (name string, message string, id int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LOCATION 'hdfs:///tmp/test/header_footer_table_1' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="2");
+
+SELECT * FROM header_footer_table_1;
+
+SELECT * FROM header_footer_table_1 WHERE id < 50;
+
+CREATE EXTERNAL TABLE header_footer_table_2 (name string, message string, id int) PARTITIONED BY (year int, month int, day int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="2");
+
+ALTER TABLE header_footer_table_2 ADD PARTITION (year=2012, month=1, day=1) location 'hdfs:///tmp/test/header_footer_table_2/2012/01/01';
+
+ALTER TABLE header_footer_table_2 ADD PARTITION (year=2012, month=1, day=2) location 'hdfs:///tmp/test/header_footer_table_2/2012/01/02';
+
+ALTER TABLE header_footer_table_2 ADD PARTITION (year=2012, month=1, day=3) location 'hdfs:///tmp/test/header_footer_table_2/2012/01/03';
+
+SELECT * FROM header_footer_table_2;
+
+SELECT * FROM header_footer_table_2 WHERE id < 50;
+
+CREATE EXTERNAL TABLE emptytable (name string, message string, id int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LOCATION 'hdfs:///tmp/test/header_footer_table_3' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="2");
+
+SELECT * FROM emptytable;
+
+SELECT * FROM emptytable WHERE id < 50;
+
+DROP TABLE header_footer_table_1;
+
+DROP TABLE header_footer_table_2;
+
+DROP TABLE emptytable;
+
+dfs -rmr hdfs:///tmp/test;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q
index f8d855b25948a190a5f910e4e05a9f52a84bbd07..7f4258f7bc15b894be277d79ccfb7c8510c0ceab 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q
@@ -12,11 +12,11 @@ create table T3 (c0 bigint,  c1 bigint, c2 int) partitioned by (ds string);
 
 create table T4 (c0 bigint, c1 string, c2 string, c3 string, c4 string, c5 string, c6 string, c7 string, c8 string, c9 string, c10 string, c11 string, c12 string, c13 string, c14 string, c15 string, c16 string, c17 string, c18 string, c19 string, c20 string, c21 string, c22 string, c23 string, c24 string, c25 string, c26 string, c27 string, c28 string, c29 string, c30 string, c31 string, c32 string, c33 string, c34 string, c35 string, c36 string, c37 string, c38 string, c39 string, c40 string, c41 string, c42 string, c43 string, c44 string, c45 string, c46 string, c47 string, c48 string, c49 string, c50 string, c51 string, c52 string, c53 string, c54 string, c55 string, c56 string, c57 string, c58 string, c59 string, c60 string, c61 string, c62 string, c63 string, c64 string, c65 string, c66 string, c67 bigint, c68 string, c69 string, c70 bigint, c71 bigint, c72 bigint, c73 string, c74 string, c75 string, c76 string, c77 string, c78 string, c79 string, c80 string, c81 bigint, c82 bigint, c83 bigint) partitioned by (ds string);
 
-insert overwrite table T1 partition (ds='2010-04-17') select '5', '1', '1', '1',  0, 0,4 from src limit 1;
+insert overwrite table T1 partition (ds='2010-04-17') select '5', '1', '1', '1',  0, 0,4 from src tablesample (1 rows);
 
-insert overwrite table T2 partition(ds='2010-04-17') select '5','name', NULL, '2', 'kavin',NULL, '9', 'c', '8', '0', '0', '7', '1','2', '0', '3','2', NULL, '1', NULL, '3','2','0','0','5','10' from src limit 1;
+insert overwrite table T2 partition(ds='2010-04-17') select '5','name', NULL, '2', 'kavin',NULL, '9', 'c', '8', '0', '0', '7', '1','2', '0', '3','2', NULL, '1', NULL, '3','2','0','0','5','10' from src tablesample (1 rows);
 
-insert overwrite table T3 partition (ds='2010-04-17') select 4,5,0 from src limit 1;
+insert overwrite table T3 partition (ds='2010-04-17') select 4,5,0 from src tablesample (1 rows);
 
 insert overwrite table T4 partition(ds='2010-04-17') 
 select 4,'1','1','8','4','5','1','0','9','U','2','2', '0','2','1','1','J','C','A','U', '2','s', '2',NULL, NULL, NULL,NULL, NULL, NULL,'1','j', 'S', '6',NULL,'1', '2', 'J', 'g', '1', 'e', '2', '1', '2', 'U', 'P', 'p', '3', '0', '0', '0', '1', '1', '1', '0', '0', '0', '6', '2', 'j',NULL, NULL, NULL,NULL,NULL, NULL, '5',NULL, 'j', 'j', 2, 2, 1, '2', '2', '1', '1', '1', '1', '1', '1', 1, 1, 32,NULL from src limit 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_numeric.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_numeric.q
new file mode 100644
index 0000000000000000000000000000000000000000..69d543f472347c634cf62db89337224a2c2ac091
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_numeric.q
@@ -0,0 +1,21 @@
+set hive.exec.dynamic.partition.mode=nonstrict;
+
+create table partint(key string, value string) partitioned by (ds string, hr int);
+insert overwrite table partint partition(ds, hr) select * from srcpart where ds = '2008-04-08';
+
+explain select key, value, hr from partint where hr < 11;
+select key, value, hr from partint where hr < 11;
+
+explain select key, value, hr from partint where hr <= 12 and hr > 11;
+select key, value, hr from partint where hr <= 12 and hr > 11;
+
+explain select key, value, hr from partint where hr between 11 and 12;
+select key, value, hr from partint where hr between 11 and 12;
+
+explain select key, value, hr from partint where hr not between 12 and 14;
+select key, value, hr from partint where hr not between 12 and 14;
+
+explain select key, value, hr from partint where hr < 13;
+select key, value, hr from partint where hr < 13;
+
+drop table partint;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q
index b76cf34120701594bbfd782731e280eae69d31be..c8a08af054579d3f5e53ead0310feb398e22418b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q
@@ -8,9 +8,9 @@ drop table gl_src_part1;
 
 
 create table gl_src1 (key int, value string) stored as textfile;
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1;
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1;
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src1;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src1;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src1;
 
 
 
@@ -49,10 +49,10 @@ select key from gl_src2 ORDER BY key ASC limit 10;
 
 -- partition
 create table gl_src_part1 (key int, value string) partitioned by (p string) stored as textfile;
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE gl_src_part1 partition(p='11');
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12');
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12');
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12');
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE gl_src_part1 partition(p='11');
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12');
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12');
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12');
 
 select key from gl_src_part1 where p like '1%' ORDER BY key ASC limit 10;
 select key from gl_src_part1 where p='11' ORDER BY key ASC limit 10;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q
index db38d43fe4354f831e26b2c70800ef8fef2c396d..7750cb90b5ca5f3bcb33e264b02326226c6141c1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q
@@ -6,7 +6,7 @@ CREATE TABLE dest1(key INT, val1 INT, val2 INT);
 CREATE TABLE dest2(key INT, val1 INT, val2 INT);
 
 CREATE TABLE INPUT(key INT, value STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv5.txt' INTO TABLE INPUT;
+LOAD DATA LOCAL INPATH '../../data/files/kv5.txt' INTO TABLE INPUT;
 
 EXPLAIN
 FROM INPUT 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby12.q
new file mode 100644
index 0000000000000000000000000000000000000000..6e9aee1d11d58ff6eb3ebb1a60f3954ee1db08de
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby12.q
@@ -0,0 +1,13 @@
+set hive.map.aggr=false;
+
+CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE;
+
+EXPLAIN
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT COUNT(src.key), COUNT(DISTINCT value) GROUP BY src.key;
+
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT COUNT(src.key), COUNT(DISTINCT value) GROUP BY src.key;
+
+SELECT dest1.* FROM dest1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q
index 1b6891e33a37e07deb0b6c10d1839fc0c8327bde..55133332a8662fee711b22751324120929cda0c3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q
@@ -5,6 +5,6 @@ CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE;
 EXPLAIN
 FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key LIMIT 5;
 
-FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key LIMIT 5;
+FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key ORDER BY src.key LIMIT 5;
 
 SELECT dest1.* FROM dest1 ORDER BY dest1.key ASC , dest1.value ASC;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q
index 82cff36422e62feaecb1c513d674f96e943216ab..dde37dfd47145e6e918e245a381cfcb89a167f59 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q
@@ -9,4 +9,4 @@ FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) G
 
 FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key;
 
-SELECT dest1.* FROM dest1;
+SELECT dest1.* FROM dest1 ORDER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q
index 874995888b447cb24786e975db0ed983734f3afb..f346cb7e90147ee477a03ef448cb943cd7c935f8 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q
@@ -9,4 +9,4 @@ FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) G
 
 FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key;
 
-SELECT dest1.* FROM dest1;
+SELECT dest1.* FROM dest1 ORDER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q
index 1b10f1e024819a6e5adf9c20824197b377037878..c587b5f658f682aea33fac1a7615e95d0bee5fad 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q
@@ -9,4 +9,4 @@ FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5))
 
 FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key;
 
-SELECT dest_g1.* FROM dest_g1;
+SELECT dest_g1.* FROM dest_g1 ORDER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q
index 4bc263c77f1b46188f588bb030306a8ba8f5b8f8..30499248cac15becd8f8d5554ef11e2643e3d4fc 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q
@@ -1,7 +1,7 @@
 set mapred.reduce.tasks=31;
 
 EXPLAIN
-SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key LIMIT 5;
+SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key ORDER BY src.key LIMIT 5;
 
-SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key LIMIT 5;
+SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key ORDER BY src.key LIMIT 5;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q
index c3cf598fb1c9002d65061f903c458411e7701f59..794ec758e9edb6f7013e3dd09d05e408f3bbd949 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q
@@ -11,4 +11,4 @@ INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(s
 FROM src
 INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) GROUP BY substr(src.key,1,1);
 
-SELECT dest1.* FROM dest1;
+SELECT dest1.* FROM dest1 ORDER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q
index 25e6789b63ef22843488e62f27b8dd56d063beab..55d1a34b3c9210ddf0248433b9f673d6a6d5ee58 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q
@@ -11,4 +11,15 @@ INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(s
 FROM src
 INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1);
 
-SELECT dest1.* FROM dest1;
+SELECT dest1.* FROM dest1 ORDER BY key;
+
+-- HIVE-5560 when group by key is used in distinct funtion, invalid result are returned
+
+EXPLAIN
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(src.key,1,1)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1);
+
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(src.key,1,1)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1);
+
+SELECT dest1.* FROM dest1 ORDER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q
index c3c82d51749fe6e62036e3ea0efa767958a6d57d..6d7cb61e2d44a95c21fa7cc243f4f1e6526a6636 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q
@@ -11,4 +11,4 @@ INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr
 FROM src
 INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) GROUP BY substr(src.key,1,1);
 
-SELECT dest_g2.* FROM dest_g2;
+SELECT dest_g2.* FROM dest_g2 ORDER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q
index b80c271afe5dc71bee606b46eed8dd15c1eedc89..b2450c9ea04e1c17c1dfe0cd527ae2c3990c8a14 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q
@@ -11,4 +11,4 @@ INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr
 FROM src
 INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1);
 
-SELECT dest_g2.* FROM dest_g2;
+SELECT dest_g2.* FROM dest_g2 ORDER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q
index 99c2d2d5a17847a7ae5d0ea3ddb5ac167f1057f8..a1ebf90aadfea84a0904402a59c4c7637ca11747 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q
@@ -12,5 +12,5 @@ INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1) GROUP BY substr(src.key,
 FROM src
 INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1) GROUP BY substr(src.key,1,1);
 
-SELECT dest1.* FROM dest1;
+SELECT dest1.* FROM dest1 ORDER BY c1;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q
index be60785d87ea525a4c86cf80eeaa6046d527f48b..e96568b398d87b0312e327625fdd04ed33839712 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q
@@ -16,5 +16,5 @@ SELECT src.key, sum(substr(src.value,5))
 FROM src
 GROUP BY src.key;
 
-SELECT dest1.* FROM dest1;
+SELECT dest1.* FROM dest1 ORDER BY key;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q
index fbf761c3aea55101ee9a17b33daad1b689645d8c..ced122fae3f5089cef0806224c131e5ad56d95dc 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q
@@ -11,6 +11,6 @@ INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1);
 FROM src
 INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1);
 
-SELECT dest1.* FROM dest1;
+SELECT dest1.* FROM dest1 ORDER BY c1;
 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q
index ac79a286055d5ec87e5dc12bbcb0d071c7b31252..0d3727b0528586f464de477aad193bcf3cf03575 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q
@@ -11,6 +11,6 @@ INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1);
 FROM src
 INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1);
 
-SELECT dest1.* FROM dest1;
+SELECT dest1.* FROM dest1 ORDER BY c1;
 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q
index 2c99d362ffff71a128d8a3b95f0aea97387e72aa..466c13222f29fa53bf38f5ddb6a20d238be053e2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q
@@ -12,6 +12,6 @@ INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1);
 FROM src
 INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1);
 
-SELECT dest1.* FROM dest1;
+SELECT dest1.* FROM dest1 ORDER BY c1;
 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q
index b1457d9349aee6663fcb0e2bf4a34489435faf10..2b8c5db41ea925de94b60104b9d3cb1c0b22fe4f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q
@@ -18,5 +18,5 @@ FROM SRC
 INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key
 INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key;
 
-SELECT DEST1.* FROM DEST1;
-SELECT DEST2.* FROM DEST2;
+SELECT DEST1.* FROM DEST1 ORDER BY key;
+SELECT DEST2.* FROM DEST2 ORDER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q
index 481b3cd084f166b9f15341513128c285e89ce156..5895ed4599849b3a2658a16837697d3930ac55a1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q
@@ -17,5 +17,5 @@ FROM SRC
 INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key
 INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key;
 
-SELECT DEST1.* FROM DEST1;
-SELECT DEST2.* FROM DEST2;
+SELECT DEST1.* FROM DEST1 ORDER BY key;
+SELECT DEST2.* FROM DEST2 ORDER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q
index a34ac8f64b11793027a643ccad1cff0618137b6e..ee6d7bf83084e591a5744e38549fbbe84aa0a2e4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q
@@ -17,5 +17,5 @@ FROM SRC
 INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key
 INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key;
 
-SELECT DEST1.* FROM DEST1;
-SELECT DEST2.* FROM DEST2;
+SELECT DEST1.* FROM DEST1 ORDER BY key;
+SELECT DEST2.* FROM DEST2 ORDER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q
index 94a3dcf7ec3f2f3213130acab295263a29b8f60d..8c2308e5d75c39282ef05e2c514c260cd636c529 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q
@@ -18,5 +18,5 @@ FROM SRC
 INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key
 INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key;
 
-SELECT DEST1.* FROM DEST1;
-SELECT DEST2.* FROM DEST2;
+SELECT DEST1.* FROM DEST1 ORDER BY key;
+SELECT DEST2.* FROM DEST2 ORDER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q
index 802aea244da9edfc166b00f73bfc49cc87f5316c..e673cc61622c8e3a8381fb4dd952c2a2610bc11d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q
@@ -14,8 +14,8 @@ INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY S
 INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key limit 10;
 
 FROM SRC
-INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key limit 10
-INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key limit 10;
+INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key ORDER BY SRC.key limit 10
+INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key ORDER BY SRC.key limit 10;
 
 SELECT DEST1.* FROM DEST1 ORDER BY key ASC, value ASC;
 SELECT DEST2.* FROM DEST2 ORDER BY key ASC, value ASC;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q
index 62b6ff5ddebb6b7199a1d22465e41b545a7968ea..0252e993363aaddc1990af742d341d1da081ff74 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q
@@ -14,6 +14,6 @@ FROM SRC
 INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key
 INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key;
 
-SELECT DEST1.* FROM DEST1;
-SELECT DEST2.* FROM DEST2;
+SELECT DEST1.* FROM DEST1 ORDER BY key;
+SELECT DEST2.* FROM DEST2 ORDER BY key;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q
index 846fd010171721714d2ad2a2d9aed43adf001634..b5e1f63a45257322092173295166520c699a83a9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q
@@ -14,6 +14,6 @@ FROM SRC
 INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key
 INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key;
 
-SELECT DEST1.* FROM DEST1;
-SELECT DEST2.* FROM DEST2;
+SELECT DEST1.* FROM DEST1 ORDER BY key;
+SELECT DEST2.* FROM DEST2 ORDER BY key;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q
index 20c8bef34223afc3104b83979cbb841f1318df9b..da85504ca18c6cdb4c92452108f3bf5bc63d5a4d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q
@@ -15,5 +15,5 @@ FROM SRC
 INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key
 INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key;
 
-SELECT DEST1.* FROM DEST1;
-SELECT DEST2.* FROM DEST2;
+SELECT DEST1.* FROM DEST1 ORDER BY key;
+SELECT DEST2.* FROM DEST2 ORDER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q
index 7e97f75cecc92a76558c4e46cf500d8808173bb7..2e3eddcb1f8662961697a9aec9533c7bfb9f3cdb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q
@@ -1,7 +1,7 @@
 set hive.map.aggr.hash.percentmemory = 0.3;
 set hive.mapred.local.mem = 384;
 
-add file ../data/scripts/dumpdata_script.py;
+add file ../../data/scripts/dumpdata_script.py;
 
 select count(distinct subq.key) from
 (FROM src MAP src.key USING 'python dumpdata_script.py' AS key WHERE src.key = 10) subq;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q
index 46e1f00d0f224465357f87f5806bbc0d4aa61394..099beb4319e0967f2fddcde8fbff1d60066e94aa 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q
@@ -3,7 +3,7 @@ set hive.groupby.skewindata=false;
 
 CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 EXPLAIN
 SELECT key, val, count(1) FROM T1 GROUP BY key, val with cube;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q
index bced21f9e494aac63e72bcba7c2cfc2f91698e17..de4a7c3cb5e5495d2bc870a73af8141f8662c0de 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q
@@ -1,6 +1,6 @@
 CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 SELECT key, val, GROUPING__ID from T1 group by key, val with cube;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q
index ffc627c82eafff6d8a9678f5c64db5fa2b875918..f451f17834502f80cf9effe2446bb29e98ceff54 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q
@@ -1,6 +1,6 @@
 CREATE TABLE T1(key INT, value INT) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/groupby_groupingid.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/groupby_groupingid.txt' INTO TABLE T1;
 
 set hive.groupby.skewindata = true;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q
index 4fba7338f1d0ae8c5f2a7266a1dd0545c169c9ae..804dfb36cf2c65a83b46877126716fe8db7691d5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q
@@ -1,6 +1,6 @@
 CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; 
 
-LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1;
 
 SELECT * FROM T1;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q
index 9f2286cc9be31e411e2665462f1e2b7f734b828a..30f1b420cc7c2a823071bcc0860c8796f1788a94 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q
@@ -2,7 +2,7 @@ set hive.new.job.grouping.set.cardinality=2;
 
 CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; 
 
-LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1;
 
 -- Since 4 grouping sets would be generated for the query below, an additional MR job should be created
 EXPLAIN
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q
index 9a00d0a7aa77f9fd20ffbd358578258bc0207f99..707737798dd6a9818bc4f486871156033e830c69 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q
@@ -4,8 +4,8 @@
 -- additional MR job is created for processing the grouping sets.
 CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; 
 
-LOAD DATA LOCAL INPATH '../data/files/grouping_sets1.txt' INTO TABLE T1;
-LOAD DATA LOCAL INPATH '../data/files/grouping_sets2.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/grouping_sets1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/grouping_sets2.txt' INTO TABLE T1;
 
 set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat;
 set hive.new.job.grouping.set.cardinality = 30;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q
index 25f1fcd793a7464b0b0897747b8c76b55425afe3..ff83185d819c5614841149e348370c83c30da1b9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q
@@ -4,7 +4,7 @@ set hive.merge.mapredfiles = false;
 
 CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; 
 
-LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1;
 
 -- This tests that cubes and rollups work fine inside sub-queries.
 EXPLAIN
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q
index fb0c5913fa07f45fa45a33bce8c684a8b2972c98..d94bd81f84f3b2ee97d0f50ebd034ceddd82a63a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q
@@ -4,7 +4,7 @@ set hive.merge.mapredfiles = false;
 
 CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; 
 
-LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1;
 
 -- This tests that cubes and rollups work fine where the source is a sub-query
 EXPLAIN
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q
index f0a8b72b0c83ec99942a13ba6af9fc834575910d..4a199365cf9687ef7bbc828104d3584bd7c2848d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q
@@ -17,4 +17,4 @@ SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(s
 WHERE src.ds = '2008-04-08'
 GROUP BY substr(src.key,1,1);
 
-SELECT dest1.* FROM dest1;
+SELECT dest1.* FROM dest1 ORDER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q
index b863344485d9f956681c740563cb2314c5c25405..cb3ee82918611f6f7dc58b170e8f49a8e988f96f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q
@@ -17,4 +17,4 @@ SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(s
 WHERE src.ds = '2008-04-08'
 GROUP BY substr(src.key,1,1);
 
-SELECT dest1.* FROM dest1;
+SELECT dest1.* FROM dest1 ORDER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_resolution.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_resolution.q
new file mode 100644
index 0000000000000000000000000000000000000000..663e33b4c7ad8708a52e0e22676220002f7dc684
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_resolution.q
@@ -0,0 +1,61 @@
+
+
+set hive.map.aggr=false;
+set hive.groupby.skewindata=false;
+explain select key, count(*) from src b group by b.key;
+explain select b.key, count(*) from src b group by key;
+
+set hive.map.aggr=false;
+set hive.groupby.skewindata=true;
+explain select key, count(*) from src b group by b.key;
+explain select b.key, count(*) from src b group by key;
+
+set hive.map.aggr=true;
+set hive.groupby.skewindata=false;
+explain select key, count(*) from src b group by b.key;
+explain select b.key, count(*) from src b group by key;
+
+set hive.map.aggr=true;
+set hive.groupby.skewindata=true;
+explain select key, count(*) from src b group by b.key;
+explain select b.key, count(*) from src b group by key;
+
+-- windowing after group by
+select key, count(*), rank() over(order by count(*))
+from src b
+where key < '12'
+group by b.key
+order by b.key;
+
+-- having after group by
+select key, count(*)
+from src b
+group by b.key
+having key < '12'
+order by b.key;
+
+-- having and windowing
+select key, count(*), rank() over(order by count(*))
+from src b
+group by b.key
+having key < '12'
+order by b.key
+;
+
+explain
+select key, count(*), rank() over(order by count(*))
+from src b
+group by b.key
+having key < '12'
+;
+
+-- order by
+select key 
+from src t 
+where key < '12'
+group by t.key 
+order by t.key;
+
+-- cluster by
+EXPLAIN
+SELECT x.key, x.value as key FROM SRC x CLUSTER BY key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q
index f79b0c472ebb1873be1404126dbf0fedcdf8e335..ee8038c7d9f5a0cbc4627776e41e7a53a0d686c7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q
@@ -3,7 +3,7 @@ set hive.groupby.skewindata=false;
 
 CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 EXPLAIN
 SELECT key, val, count(1) FROM T1 GROUP BY key, val with rollup;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q
index 911a11ae8990f10f368f63e576a7de5cedd7e8b7..7401a9ca1d9bd1410e6ba44bda9819be9b90afbe 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q
@@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true;
 CREATE TABLE T1(key STRING, val STRING)
 CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 -- perform an insert to make sure there are 2 files
 INSERT OVERWRITE TABLE T1 select key, val from T1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q
index 31b4ec5c74dc24df144d94eaf458ecfc0a696396..700a8af91548e599e765bf7f387b5389c5214657 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q
@@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true;
 CREATE TABLE T1(key STRING, val STRING)
 CLUSTERED BY (key) SORTED BY (val) INTO 2 BUCKETS STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 -- perform an insert to make sure there are 2 files
 INSERT OVERWRITE TABLE T1 select key, val from T1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q
index 103c57a123576175cbc2e09721eceae291f79cd0..2ef8447935a66ec6c9db0b81e65c4e2f28363b96 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q
@@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true;
 CREATE TABLE T1(key STRING, val STRING)
 CLUSTERED BY (key) SORTED BY (key, val) INTO 2 BUCKETS STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 -- perform an insert to make sure there are 2 files
 INSERT OVERWRITE TABLE T1 select key, val from T1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q
index e43da3c93225fdaa6435b157e2a109f117755a3a..3c959e381f2201cfa2629cdc91e48750d081f4e9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q
@@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true;
 CREATE TABLE T1(key STRING, val STRING)
 CLUSTERED BY (key, val) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 -- perform an insert to make sure there are 2 files
 INSERT OVERWRITE TABLE T1 select key, val from T1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q
index bef5e5d2d547a5c94601cdff4ffcc49eda4349dd..dd05238f1ccc4fa269b705df379e6261a4a5446d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q
@@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true;
 CREATE TABLE T1(key STRING, val STRING)
 CLUSTERED BY (val) SORTED BY (key, val) INTO 2 BUCKETS STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 -- perform an insert to make sure there are 2 files
 INSERT OVERWRITE TABLE T1 select key, val from T1;
@@ -30,7 +30,7 @@ DROP TABLE T1;
 CREATE TABLE T1(key STRING, val STRING)
 CLUSTERED BY (val, key) SORTED BY (key, val) INTO 2 BUCKETS STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 -- perform an insert to make sure there are 2 files
 INSERT OVERWRITE TABLE T1 select key, val from T1;
@@ -52,7 +52,7 @@ DROP TABLE T1;
 CREATE TABLE T1(key STRING, val STRING)
 CLUSTERED BY (val) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 -- perform an insert to make sure there are 2 files
 INSERT OVERWRITE TABLE T1 select key, val from T1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q
index cf076e8125612b0ccfcab686088fa19383b1df63..aa09aec34b2336948ea3bc5411b5ac9abd9ae2bb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q
@@ -17,7 +17,7 @@ SELECT key, count(1) FROM T1 where ds = '1' GROUP BY key;
 
 SELECT * FROM outputTbl1 ORDER BY key;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='2');
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='2');
 
 -- The plan should not be converted to a map-side group since no partition is being accessed
 EXPLAIN EXTENDED
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q
index c2d42154e516cc4a9ff687e6d9b123f187e73dfc..99337859fb00b7a078c302396f0cda4b516e296c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q
@@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true;
 CREATE TABLE T1(key STRING, val STRING) PARTITIONED BY (ds string)
 CLUSTERED BY (val) SORTED BY (key, val) INTO 2 BUCKETS STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1');
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1');
 
 -- perform an insert to make sure there are 2 files
 INSERT OVERWRITE TABLE T1 PARTITION (ds='1') select key, val from T1 where ds = '1';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q
index 121804e60a9dad81e5e2c0167f819430cf66224e..f53295e4b243512b338f675b27d8a31a00178e30 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q
@@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true;
 CREATE TABLE T1(key STRING, val STRING) PARTITIONED BY (ds string)
 CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1');
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1');
 
 -- perform an insert to make sure there are 2 files
 INSERT OVERWRITE TABLE T1 PARTITION (ds='1') select key, val from T1 where ds = '1';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q
index 1c3d1cdcc265d038a1e4083355c1fcda582ff8aa..296336d0f9f1a812866df146b8f2fc2cccbcc4df 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q
@@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true;
 CREATE TABLE T1(key STRING, val STRING) PARTITIONED BY (ds string)
 CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1');
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1');
 
 -- perform an insert to make sure there are 2 files
 INSERT OVERWRITE TABLE T1 PARTITION (ds='1') select key, val from T1 where ds = '1';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q
index 068c26a9c2b4b5171c8e1a83fee5796abeb2af0b..db0faa04da0ec59943388974fc2cc33382dff274 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q
@@ -7,7 +7,7 @@ set hive.groupby.skewindata=true;
 CREATE TABLE T1(key STRING, val STRING)
 CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 -- perform an insert to make sure there are 2 files
 INSERT OVERWRITE TABLE T1 select key, val from T1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q
index 8efa05e2544d4720cb29a713f8fbbce8828b4850..4ec138e51a80694e5d4b98ec59c14498a7d87202 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q
@@ -7,7 +7,7 @@ set hive.map.groupby.sorted.testmode=true;
 CREATE TABLE T1(key STRING, val STRING)
 CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 -- perform an insert to make sure there are 2 files
 INSERT OVERWRITE TABLE T1 select key, val from T1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/import_exported_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/import_exported_table.q
new file mode 100644
index 0000000000000000000000000000000000000000..cb147c5feab24d2badf7b1f0f10f946444ecdcfb
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/import_exported_table.q
@@ -0,0 +1,13 @@
+dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_import_exported_table/;
+dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_import_exported_table/exported_table/;
+dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_import_exported_table/exported_table/data/;
+
+dfs -copyFromLocal ../../data/files/exported_table/_metadata hdfs:///tmp/test_import_exported_table/exported_table;
+dfs -copyFromLocal ../../data/files/exported_table/data/data hdfs:///tmp/test_import_exported_table/exported_table/data;
+
+IMPORT FROM '/tmp/test_import_exported_table/exported_table';
+DESCRIBE j1_41;
+SELECT * from j1_41;
+
+dfs -rmr hdfs:///tmp/test_import_exported_table;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q
index 33a1fc581ed13b1d4649317c57beaa6e14ce1980..03d77f1f19b01be62ed78ad36a68095d5736a915 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q
@@ -1,15 +1,18 @@
+set hive.stats.dbclass=fs;
 SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 create table foobar(key int, value string) PARTITIONED BY (ds string, hr string);
 alter table foobar add partition (ds='2008-04-08',hr='12');
 
-CREATE INDEX srcpart_auth_index ON TABLE foobar(key) as 'BITMAP' WITH DEFERRED REBUILD;
+CREATE INDEX srcpart_AUTH_index ON TABLE foobar(key) as 'BITMAP' WITH DEFERRED REBUILD;
+SHOW INDEXES ON foobar;
+
 grant select on table foobar to user hive_test_user;
-grant select on table default__foobar_srcpart_auth_index__ to user hive_test_user;
-grant update on table default__foobar_srcpart_auth_index__ to user hive_test_user;
-grant create on table default__foobar_srcpart_auth_index__ to user hive_test_user;
+grant select on table default__foobar_srcpart_auth_indeX__ to user hive_test_user;
+grant update on table default__foobar_srcpart_auth_indEx__ to user hive_test_user;
+grant create on table default__foobar_srcpart_auth_inDex__ to user hive_test_user;
 set hive.security.authorization.enabled=true;
 
-ALTER INDEX srcpart_auth_index ON foobar PARTITION (ds='2008-04-08',hr='12')  REBUILD;
+ALTER INDEX srcpart_auth_INDEX ON foobar PARTITION (ds='2008-04-08',hr='12')  REBUILD;
 set hive.security.authorization.enabled=false;
 DROP INDEX srcpart_auth_index on foobar;
 DROP TABLE foobar;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q
index cb8a1d6293f8d576962b9f7c12e9fd74978cdacd..77733aac02686315a5cd2f7045c09351e80025a5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q
@@ -3,6 +3,7 @@
 -- without indexing
 SELECT key, value FROM src WHERE key > 80 AND key < 100 ORDER BY key;
 
+set hive.stats.dbclass=fs;
 CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD;
 ALTER INDEX src_index ON src REBUILD;
 
@@ -25,4 +26,4 @@ SET hive.optimize.index.filter.compact.minsize=0;
 EXPLAIN SELECT key, value FROM src WHERE key > 80 AND key < 100 ORDER BY key;
 SELECT key, value FROM src WHERE key > 80 AND key < 100 ORDER BY key;
 
-DROP INDEX src_index on src;
\ No newline at end of file
+DROP INDEX src_index on src;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q
index cb32162d40a06b32f8fe2a57e3d8e321ee9b4224..41f4a40823e4d9542452f1736475ea6778da6d1a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q
@@ -3,6 +3,7 @@
 -- Create temp, and populate it with some values in src.
 CREATE TABLE temp(key STRING, val STRING) STORED AS TEXTFILE;
 
+set hive.stats.dbclass=fs;
 -- Build an index on temp.
 CREATE INDEX temp_index ON TABLE temp(key) as 'COMPACT' WITH DEFERRED REBUILD;
 ALTER INDEX temp_index ON temp REBUILD;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q
index 790e6c223f7468c15f24ac39d2d4fd3643a671c5..2967bd60d8bc65b5eee60dcba5ac2b2260883d33 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 -- test automatic use of index on different file formats
 CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD;
 ALTER INDEX src_index ON src REBUILD;
@@ -16,4 +17,4 @@ SET hive.optimize.index.filter.compact.minsize=0;
 EXPLAIN SELECT key, value FROM src WHERE key=86 ORDER BY key;
 SELECT key, value FROM src WHERE key=86 ORDER BY key;
 
-DROP INDEX src_index on src;
\ No newline at end of file
+DROP INDEX src_index on src;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q
index 2bf8481f1d6e259a7a4737093037a663e2f16224..a672e06e79332465bb49a7b1ab70e6cc17d4af31 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q
@@ -4,6 +4,7 @@
 EXPLAIN SELECT a.key, a.value FROM src a JOIN srcpart b ON (a.key = b.key) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key;
 SELECT a.key, a.value FROM src a JOIN srcpart b ON (a.key = b.key) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key;
 
+set hive.stats.dbclass=fs;
 
 CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD;
 ALTER INDEX src_index ON src REBUILD;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q
index 808a04cc360a6f54a62a41f11c13ec53770a3deb..d78e0fd58a07460be3a1fc4fcc6c7a9b63c6eb8f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q
@@ -4,6 +4,7 @@
 EXPLAIN SELECT a.key, a.value FROM src a JOIN srcpart b ON (a.key = b.key) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key;
 SELECT a.key, a.value FROM src a JOIN srcpart b ON (a.key = b.key) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key;
 
+set hive.stats.dbclass=fs;
 
 CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD;
 ALTER INDEX src_index ON src REBUILD;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q
index 06e97fa76bc973da7cbd5539320e2fc63d73e1f9..f0a91b4b8a5928b30a0a80142da0f86ce6acd845 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 -- With multiple indexes, make sure we choose which to use in a consistent order
 
 CREATE INDEX src_key_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD;
@@ -13,4 +14,4 @@ EXPLAIN SELECT key, value FROM src WHERE key=86 ORDER BY key;
 SELECT key, value FROM src WHERE key=86 ORDER BY key;
 
 DROP INDEX src_key_index ON src;
-DROP INDEX src_val_index ON src;
\ No newline at end of file
+DROP INDEX src_val_index ON src;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q
index 5013d29e732c472f1d8535742079fa0240eca889..70166b36c5f3c7b6f1b0455261dd9898a451b985 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 -- test automatic use of index on table with partitions
 CREATE INDEX src_part_index ON TABLE srcpart(key) as 'COMPACT' WITH DEFERRED REBUILD;
 ALTER INDEX src_part_index ON srcpart REBUILD;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q
index 0984a4a21ba17316c655a847da596d346b6ceeca..1d9efbbc6d738c8e7b94bbbdd53cf11f51892382 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q
@@ -3,6 +3,7 @@
 EXPLAIN SELECT a.key, b.key FROM src a JOIN src b ON (a.value = b.value) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key;
 SELECT a.key, b.key FROM src a JOIN src b ON (a.value = b.value) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key;
 
+set hive.stats.dbclass=fs;
 CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD;
 ALTER INDEX src_index ON src REBUILD;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q
index d8f3eda1813c8c5c1eda9e52f03f0d0dd9f05961..acd4194b0e7a4115addbb1b65d29ccd0d1878d66 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 -- test cases where the index should not be used automatically
 
 CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q
index f9deb2883960c774d8f815495cfbe47b4fb49330..673c835fb9084fc83d8eb622872426e368c1d61f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 DROP INDEX srcpart_index_proj on srcpart;
 
 EXPLAIN
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q
index 55633d9225fa79db8f09b648f2445fd80dce6538..adec8f1b3bfb770419a0f1d93a4bc93e311a395c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 EXPLAIN
 CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD;
 CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q
index bd15a21fa420085eaf4ea139e764dc857d3dd20e..1ffa6eeebbfb37e2eff5494b97c66fd829957c8c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 EXPLAIN
 CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD;
 EXPLAIN
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q
index 0d2c811459dcb36a98c9f9874dc8df14d5b1d465..e7a093c118ba0fb9129d3eeaec0c24a3c36e90c5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q
@@ -1,3 +1,6 @@
+set hive.stats.dbclass=counter;
+set hive.stats.autogather=true;
+
 EXPLAIN
 CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD;
 EXPLAIN
@@ -17,10 +20,10 @@ SELECT a.bucketname AS `_bucketname`, COLLECT_SET(a.offset) as `_offsets`
 FROM (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src1_index__
         WHERE key = 0) a
   JOIN 
-     (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__
-        WHERE value = "val_0") b
-  ON
-    a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT
+    (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__
+       WHERE value = "val_0") b
+ ON
+   a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT
 EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname;
 
 INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q
index 672ce29f1bb47fd696b857b8549ddad79bbc65af..56cd44dd5b3239f7ee9289870e8e3c70838048ef 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q
@@ -1,3 +1,6 @@
+set hive.stats.dbclass=counter;
+set hive.stats.autogather=true;
+
 -- try the query without indexing, with manual indexing, and with automatic indexing
 -- without indexing
 SELECT key, value FROM src WHERE key=0 AND value = "val_0" ORDER BY key;
@@ -21,12 +24,12 @@ SELECT * FROM default__src_src2_index__ ORDER BY value;
 EXPLAIN
 SELECT a.bucketname AS `_bucketname`, COLLECT_SET(a.offset) as `_offsets`
 FROM (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src1_index__
-        WHERE key = 0) a
-  JOIN 
-     (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__
-        WHERE value = "val_0") b
-  ON
-    a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT
+       WHERE key = 0) a
+ JOIN 
+    (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__
+       WHERE value = "val_0") b
+ ON
+   a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT
 EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname;
 
 INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q
index 90d7987594bac5b168ab91a504abeb42f40a339d..3b310cee4cb293a301a5f2f53a240f2271425561 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 -- test automatic use of index on table with partitions
 CREATE INDEX src_part_index ON TABLE srcpart(key) as 'BITMAP' WITH DEFERRED REBUILD;
 ALTER INDEX src_part_index ON srcpart REBUILD;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q
index 2f5e5d4fb231bab8d7c40522aec623f7fe59a5f9..32ecfb9db8a0ead4f86113ea5d11696cfe259d7f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 SET hive.exec.compress.result=true;
 CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD;
 ALTER INDEX src_index ON src REBUILD;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q
index 054df51c32180cbc75fd902343b23751764957ef..26a351ea31858bc0695855d05d9e24a75b93ea6d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 CREATE TABLE srcpart_rc (key int, value string) PARTITIONED BY (ds string, hr int) STORED AS RCFILE;
 
 INSERT OVERWRITE TABLE srcpart_rc PARTITION (ds='2008-04-08', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 11;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact.q
index a936f1127f69071f45e444809541f3babb7e0602..98cbec147db518bda030e0c5a325bc420d4b1159 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 DROP INDEX srcpart_index_proj on srcpart;
 
 EXPLAIN
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q
index 837033be403a973ce38347808318ba6efd70718a..97276f488e742a61b6ac2c1a172115d06be51e0f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 EXPLAIN
 CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD;
 CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q
index 56119ac11f32f0188fb5cca62e3ee0a81d7b371b..1eb3f5c3dbcbfc95af8d64443137c88993ff629e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 CREATE TABLE srcpart_rc (key int, value string) PARTITIONED BY (ds string, hr int) STORED AS RCFILE;
 
 INSERT OVERWRITE TABLE srcpart_rc PARTITION (ds='2008-04-08', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 11;
@@ -42,4 +43,4 @@ SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 SELECT key, value FROM srcpart_rc WHERE key=100 ORDER BY key;
 
 DROP INDEX srcpart_rc_index on srcpart_rc;
-DROP TABLE srcpart_rc;
\ No newline at end of file
+DROP TABLE srcpart_rc;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q
index f3fcb4af3d5e6b1cdd066b62b4d208a0036e6d3c..599b4ac1149ecb07e11bae7eee6ea40a1ebe126a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 CREATE TABLE src_index_test_rc (key int, value string) STORED AS RCFILE;
 
 INSERT OVERWRITE TABLE src_index_test_rc SELECT * FROM src;
@@ -16,4 +17,4 @@ SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 SELECT key, value FROM src_index_test_rc WHERE key=100 ORDER BY key;
 
 DROP INDEX src_index on src_index_test_rc;
-DROP TABLE src_index_test_rc;
\ No newline at end of file
+DROP TABLE src_index_test_rc;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q
index d0d9a32adcba1d881a645a5b8573f8fea52adb06..e72b27c781a80b06e43161f1cbdfebc80a32536a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q
@@ -1,6 +1,6 @@
 SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 SET hive.default.fileformat=TextFile;
-
+set hive.stats.dbclass=fs;
 CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD;
 ALTER INDEX src_index ON src REBUILD;
 
@@ -129,4 +129,4 @@ SELECT * FROM src WHERE key >= '9';
 
 SET hive.exec.post.hooks=;
 
-DROP INDEX src_index ON src;
\ No newline at end of file
+DROP INDEX src_index ON src;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q
index 84ed3cc932e8c2a67fd89fa648a5f9878765d8ab..963b8f74e5f07fa38d15b629b646e3a77a334c31 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q
@@ -1,4 +1,5 @@
 SET hive.exec.compress.result=true;
+set hive.stats.dbclass=fs;
 CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD;
 ALTER INDEX src_index ON src REBUILD;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q
index 062821e870dce0e79094d8a1ee4471ddd815a916..ef020b63d5a72e2d9f11f0b76d96c5bee570c968 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 drop index src_index_2 on src;
 drop index src_index_3 on src;
 drop index src_index_4 on src;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q
index a6fe16ba3b809c05f11b7a203490d0794650421a..20186a7400494f4ff84cbec5a9204f3ee173e144 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 -- Want to ensure we can build and use indices on tables stored with SerDes
 -- Build the (Avro backed) table
 CREATE TABLE doctors 
@@ -31,7 +32,7 @@ TBLPROPERTIES ('avro.schema.literal'='{
 
 DESCRIBE doctors;
 
-LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors;
+LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors;
 
 -- Create and build an index
 CREATE INDEX doctors_index ON TABLE doctors(number) AS 'COMPACT' WITH DEFERRED REBUILD;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q
index 82e15b97b6ac463ec6cc5ae35bdb5b613c08901a..ecab2b7f6b3321cb17028fadf81b25b4f3fc109b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 -- test that stale indexes are not used
 
 CREATE TABLE temp(key STRING, val STRING) STORED AS TEXTFILE;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q
index e7cfeff31f6dc8e691849f2f51d98930343e5c23..a93ccf7f95fc3b2deef1c9d55cb6a42437f36b75 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 -- Test if index is actually being used.
 
 -- Create temp, and populate it with some values in src.
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q
index 119994e91b056d5668f7240106093ff7b09c6cb3..728b8cc4a949741950896a18cbecb238cb8aca27 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q
@@ -47,12 +47,12 @@ CREATE TABLE srcpart_merge_dp LIKE srcpart;
 CREATE TABLE srcpart_merge_dp_rc LIKE srcpart;
 ALTER TABLE srcpart_merge_dp_rc SET FILEFORMAT RCFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11);
-LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11);
-LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11);
-LOAD DATA LOCAL INPATH '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11);
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11);
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11);
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11);
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11);
 
-LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=12);
+LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=12);
 
 INSERT OVERWRITE TABLE srcpart_merge_dp_rc PARTITION (ds = '2008-04-08', hr) 
 SELECT key, value, hr FROM srcpart_merge_dp WHERE ds = '2008-04-08';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q
index a039dc5a2a0401786c28a72adb3a7da4d96323b7..ce5ed8419dd444770c085c780ff29b83922f6e71 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q
@@ -1,7 +1,7 @@
 DROP TABLE infertypes;
 CREATE TABLE infertypes(ti TINYINT, si SMALLINT, i INT, bi BIGINT, fl FLOAT, db DOUBLE, str STRING);
 
-LOAD DATA LOCAL INPATH '../data/files/infer_const_type.txt' OVERWRITE INTO TABLE infertypes;
+LOAD DATA LOCAL INPATH '../../data/files/infer_const_type.txt' OVERWRITE INTO TABLE infertypes;
 
 SELECT * FROM infertypes;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q
index 40fbc84a981f1cbf2a10d673dad370f3e5c889f1..620e9dcfae6cf6c47f8b755ea080eb700d972937 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q
@@ -7,15 +7,15 @@ FROM src
 INSERT OVERWRITE TABLE dest1 SELECT src.* WHERE src.key < 100
 INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key >= 100 and src.key < 200
 INSERT OVERWRITE TABLE dest3 PARTITION(ds='2008-04-08', hr='12') SELECT src.key WHERE src.key >= 200 and src.key < 300
-INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300;
+INSERT OVERWRITE DIRECTORY 'target/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300;
 
 FROM src
 INSERT OVERWRITE TABLE dest1 SELECT src.* WHERE src.key < 100
 INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key >= 100 and src.key < 200
 INSERT OVERWRITE TABLE dest3 PARTITION(ds='2008-04-08', hr='12') SELECT src.key WHERE src.key >= 200 and src.key < 300
-INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300;
+INSERT OVERWRITE DIRECTORY 'target/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300;
 
 SELECT dest1.* FROM dest1;
 SELECT dest2.* FROM dest2;
 SELECT dest3.* FROM dest3;
-dfs -cat ../build/ql/test/data/warehouse/dest4.out/*;
+dfs -cat ${system:test.warehouse.dir}/dest4.out/*;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q
index 82e6d81426efce926c51bce7a3fc277cf5683f02..4990d0ba1cd54a7748877909f08b3d95d455406f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q
@@ -1,6 +1,6 @@
 -- TestSerDe is a user defined serde where the default delimiter is Ctrl-B
 DROP TABLE INPUT16;
-ADD JAR ../data/files/TestSerDe.jar;
+ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar;
 CREATE TABLE INPUT16(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1_cb.txt' INTO TABLE INPUT16;
+LOAD DATA LOCAL INPATH '../../data/files/kv1_cb.txt' INTO TABLE INPUT16;
 SELECT INPUT16.VALUE, INPUT16.KEY FROM INPUT16;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q
index 5dab4103d8dda8d0eec7f3d9aa1eb20718853464..9272a92c8102e8d565c0ff02dd40f8c8cfdc76ab 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q
@@ -4,8 +4,8 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
 -- the user is overwriting it with ctrlC
 
 DROP TABLE INPUT16_CC;
-ADD JAR ../data/files/TestSerDe.jar;
+ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar;
 CREATE TABLE INPUT16_CC(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe'  with serdeproperties ('testserde.default.serialization.format'='\003', 'dummy.prop.not.used'='dummyy.val') STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1_cc.txt' INTO TABLE INPUT16_CC;
+LOAD DATA LOCAL INPATH '../../data/files/kv1_cc.txt' INTO TABLE INPUT16_CC;
 SELECT INPUT16_CC.VALUE, INPUT16_CC.KEY FROM INPUT16_CC;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q
index fec44e97669dd99f15b7a82c29e0e4239b3938d0..3dc7fec9f66698cfb93bc28d8511a4bbad5a2506 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q
@@ -1,5 +1,5 @@
 
 create table apachelog(ipaddress STRING,identd STRING,user_name STRING,finishtime STRING,requestline string,returncode INT,size INT) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.dynamic_type.DynamicSerDe' WITH SERDEPROPERTIES (  'serialization.format'= 'org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol',  'quote.delim'= '("|\\[|\\])',  'field.delim'=' ',  'serialization.null.format'='-'  ) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/apache.access.log' INTO TABLE apachelog;
+LOAD DATA LOCAL INPATH '../../data/files/apache.access.log' INTO TABLE apachelog;
 SELECT a.* FROM apachelog a;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q
index 0566ab18c2537541fafe9c35aeac3e86cc094801..ff430abb8e8aa942d0a9457b9be2757be55fa3b1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q
@@ -1,6 +1,6 @@
 CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE;
 
-ADD FILE ../data/scripts/input20_script;
+ADD FILE ../../data/scripts/input20_script.py;
 
 EXPLAIN
 FROM (
@@ -12,7 +12,7 @@ FROM (
 ) tmap
 INSERT OVERWRITE TABLE dest1
 REDUCE tmap.key, tmap.value
-USING 'input20_script'
+USING 'python input20_script.py'
 AS key, value;
 
 FROM (
@@ -24,7 +24,7 @@ FROM (
 ) tmap
 INSERT OVERWRITE TABLE dest1
 REDUCE tmap.key, tmap.value
-USING 'input20_script'
+USING 'python input20_script.py'
 AS key, value;
 
 SELECT * FROM dest1 SORT BY key, value;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q
index d7c814e58061c2c93c5d88b1a7a525e3628c2a76..43cd01e684b370e464c413639cd954f91f37e29a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q
@@ -1,7 +1,7 @@
 
 
 CREATE TABLE src_null(a STRING, b STRING, c STRING, d STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/null.txt' INTO TABLE src_null;
+LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE src_null;
 
 EXPLAIN SELECT * FROM src_null DISTRIBUTE BY c SORT BY d;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q
index 853947be57a1f429f3cc1787897419aaabdd7497..8803e4dbeb8cf5bec6c784ab6d73a20503e89afb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q
@@ -1,5 +1,5 @@
 CREATE TABLE INPUT4(KEY STRING, VALUE STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUT4;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUT4;
 
 EXPLAIN
 SELECT a.KEY2
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q
index 7ab17515af5c72c465dc63ed5b12730db1fa0f40..8b6b21502001e058caa978e0e91ea048c97582a4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q
@@ -1,6 +1,6 @@
 CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE;
 
-ADD FILE ../data/scripts/input20_script;
+ADD FILE ../../data/scripts/input20_script.py;
 
 EXPLAIN
 FROM (
@@ -12,7 +12,7 @@ FROM (
 ) tmap
 INSERT OVERWRITE TABLE dest1
 REDUCE tmap.key, tmap.value
-USING 'input20_script'
+USING 'python input20_script.py'
 AS (key STRING, value STRING);
 
 FROM (
@@ -24,7 +24,7 @@ FROM (
 ) tmap
 INSERT OVERWRITE TABLE dest1
 REDUCE tmap.key, tmap.value
-USING 'input20_script'
+USING 'python input20_script.py'
 AS (key STRING, value STRING);
 
 SELECT * FROM dest1 SORT BY key, value;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q
index 6fd136afec4440baea47c209f073db304cf404e7..6ded61aa23990b41c6b6c18f534624caba081cd2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q
@@ -1,6 +1,6 @@
 create table documents(contents string) stored as textfile;
 
-LOAD DATA LOCAL INPATH '../data/files/docurl.txt' INTO TABLE documents;
+LOAD DATA LOCAL INPATH '../../data/files/docurl.txt' INTO TABLE documents;
 
 
 select url, count(1) 
@@ -8,7 +8,7 @@ FROM
 (
   FROM documents
   MAP documents.contents
-  USING 'java -cp ../build/ql/test/classes org.apache.hadoop.hive.scripts.extracturl' AS (url, count)
+  USING 'java -cp ../util/target/classes/ org.apache.hadoop.hive.scripts.extracturl' AS (url, count)
 ) subq
 group by url;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q
index 3584820acaf4af039f58012e275136f5897b16a4..f983aca847d9576c959ae07b341f4bef8e7ce6ad 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q
@@ -1,7 +1,7 @@
 
 CREATE TABLE T1(key STRING, value STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1;
-LOAD DATA LOCAL INPATH '../data/files/kv2.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/kv2.txt' INTO TABLE T1;
 
 
 CREATE TABLE T2(key STRING, value STRING);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q
index 08d6d97603045233b90e1c5298e212fe23ec4642..1186bbbbe6bbd4542005fbe4060928537e2892aa 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q
@@ -1,7 +1,7 @@
 CREATE TABLE INPUT4(KEY STRING, VALUE STRING) STORED AS TEXTFILE;
 EXPLAIN
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUT4;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUT4;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUT4;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUT4;
 EXPLAIN FORMATTED
 SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias;
 SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q
index 4166cb5f94939558fa48fe34894303ab8f58e4a8..ab187b5d7e7d1f47da96117a8d5a321bd5e40be5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q
@@ -2,15 +2,15 @@
 
 
 create table tmp_insert_test (key string, value string) stored as textfile;
-load data local inpath '../data/files/kv1.txt' into table tmp_insert_test;
+load data local inpath '../../data/files/kv1.txt' into table tmp_insert_test;
 select * from tmp_insert_test;
 
 create table tmp_insert_test_p (key string, value string) partitioned by (ds string) stored as textfile;
 
-load data local inpath '../data/files/kv1.txt' into table tmp_insert_test_p partition (ds = '2009-08-01');
+load data local inpath '../../data/files/kv1.txt' into table tmp_insert_test_p partition (ds = '2009-08-01');
 select * from tmp_insert_test_p where ds= '2009-08-01'
 order by key, value;
 
-load data local inpath '../data/files/kv2.txt' into table tmp_insert_test_p partition (ds = '2009-08-01');
+load data local inpath '../../data/files/kv2.txt' into table tmp_insert_test_p partition (ds = '2009-08-01');
 select * from tmp_insert_test_p where ds= '2009-08-01'
 order by key, value;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q
index 5512dc328065d2b4f234791f3f89721f8caab48d..3182bbef3981258b353113a81fa0eb61e3a7c237 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q
@@ -1,7 +1,7 @@
 drop table tst_src1;
 create table tst_src1 like src1;
-load data local inpath '../data/files/kv1.txt' into table tst_src1 ;
+load data local inpath '../../data/files/kv1.txt' into table tst_src1 ;
 select count(1) from tst_src1;
-load data local inpath '../data/files/kv1.txt' into table tst_src1 ;
+load data local inpath '../../data/files/kv1.txt' into table tst_src1 ;
 select count(1) from tst_src1;
 drop table tst_src1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q
index 4557edc178b7fd05b3334e0dca20722d7a80967e..2e975e58c1e93c7eb816d276921164f7c2bdd616 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q
@@ -3,4 +3,4 @@ CREATE TABLE dest(key INT, value STRING) STORED AS TEXTFILE;
 SET hive.output.file.extension=.txt;
 INSERT OVERWRITE TABLE dest SELECT src.* FROM src;
 
-dfs -cat ../build/ql/test/data/warehouse/dest/*.txt
\ No newline at end of file
+dfs -cat ${system:test.warehouse.dir}/dest/*.txt
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q
index 633a8c6edb5cc371e55337f2a4a97377052ff2fa..334da264d6e55f0f66ec5996d7be205b43f8a093 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q
@@ -2,8 +2,8 @@ SET hive.insert.into.multilevel.dirs=true;
 
 SET hive.output.file.extension=.txt;
 
-INSERT OVERWRITE DIRECTORY '../build/ql/test/data/x/y/z/' SELECT src.* FROM src;
+INSERT OVERWRITE DIRECTORY 'target/data/x/y/z/' SELECT src.* FROM src;
 
-dfs -cat ../build/ql/test/data/x/y/z/*.txt;
+dfs -cat ${system:build.dir}/data/x/y/z/*.txt;
 
-dfs -rmr ../build/ql/test/data/x;
\ No newline at end of file
+dfs -rmr ${system:build.dir}/data/x;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q
index 8c57dd3f25aa6e8841dfb0111cdb0230af412e99..b18d60aa74e730a7ec16e86d515b2b532b47a582 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q
@@ -1,4 +1,4 @@
 CREATE TABLE INPUT4_CB(KEY STRING, VALUE STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' LINES TERMINATED BY '\012' STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1_cb.txt' INTO TABLE INPUT4_CB;
+LOAD DATA LOCAL INPATH '../../data/files/kv1_cb.txt' INTO TABLE INPUT4_CB;
 SELECT INPUT4_CB.VALUE, INPUT4_CB.KEY FROM INPUT4_CB;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q
index 4f5824df5c9d557ab136696278302d7182d64c67..b108cbd6b23e5c59cebbf00e3bd1b16ce438734f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q
@@ -1,2 +1,2 @@
-dfs -cat ../data/files/kv1.txt;
+dfs -cat ../../data/files/kv1.txt;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q
index 9a7ca5da1277f987b012ef45ccb698102fc57d6d..87c55a26d7b83d3076b78b6d9792de3a12c688e5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q
@@ -1,7 +1,7 @@
 -- test for internationalization
 -- kv4.txt contains the utf-8 character 0xE982B5E993AE which we are verifying later on
 CREATE TABLE INPUTDDL5(name STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv4.txt' INTO TABLE INPUTDDL5;
+LOAD DATA LOCAL INPATH '../../data/files/kv4.txt' INTO TABLE INPUTDDL5;
 DESCRIBE INPUTDDL5;
 SELECT INPUTDDL5.name from INPUTDDL5;
 SELECT count(1) FROM INPUTDDL5 WHERE INPUTDDL5.name = _UTF-8 0xE982B5E993AE;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q
index d33ab8d9a78f41f1bb51fc78103840337ff4b9ae..6c709399a3c0d439fe7f3084eae8f9a16e7b954e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q
@@ -2,8 +2,8 @@
 -- test for describe extended table partition
 -- test for alter table drop partition
 CREATE TABLE INPUTDDL6(KEY STRING, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUTDDL6 PARTITION (ds='2008-04-09');
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUTDDL6 PARTITION (ds='2008-04-08');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUTDDL6 PARTITION (ds='2008-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUTDDL6 PARTITION (ds='2008-04-08');
 DESCRIBE EXTENDED INPUTDDL6;
 DESCRIBE EXTENDED INPUTDDL6 PARTITION (ds='2008-04-08');
 SHOW PARTITIONS INPUTDDL6;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q
index 8a73935feec51ce0d7d05164e6d9ae3f39807846..27e587a283cd3cec5f3cc1c392ca0a635a8620eb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q
@@ -3,22 +3,22 @@
 
 
 CREATE TABLE T1(name STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T1;
 SELECT COUNT(1) FROM T1;
 
 
 CREATE TABLE T2(name STRING) STORED AS SEQUENCEFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.seq' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T2;
 SELECT COUNT(1) FROM T2;
 
 
 CREATE TABLE T3(name STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T3 PARTITION (ds='2008-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T3 PARTITION (ds='2008-04-09');
 SELECT COUNT(1) FROM T3 where T3.ds='2008-04-09';
 
 
 CREATE TABLE T4(name STRING) PARTITIONED BY(ds STRING) STORED AS SEQUENCEFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.seq' INTO TABLE T4 PARTITION (ds='2008-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T4 PARTITION (ds='2008-04-09');
 SELECT COUNT(1) FROM T4 where T4.ds='2008-04-09';
 
 DESCRIBE EXTENDED T1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q
index 6ad70b5673f30739da77427f0f8d7fb21ca8f291..6b00f977c4c68bd3131bb326be682e32a24ade63 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q
@@ -1,8 +1,8 @@
 CREATE TABLE sourceTable (one string,two string) PARTITIONED BY (ds string,hr string);
 
-load data local inpath '../data/files/kv1.txt' INTO TABLE sourceTable partition(ds='2011-11-11', hr='11');
+load data local inpath '../../data/files/kv1.txt' INTO TABLE sourceTable partition(ds='2011-11-11', hr='11');
 
-load data local inpath '../data/files/kv3.txt' INTO TABLE sourceTable partition(ds='2011-11-11', hr='12');
+load data local inpath '../../data/files/kv3.txt' INTO TABLE sourceTable partition(ds='2011-11-11', hr='12');
 
 CREATE TABLE destinTable (one string,two string) PARTITIONED BY (ds string,hr string);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q
index 598d30eaebba2cb973774daaed35a0dc1ab91a48..bd1eb752879e6b8866e26e1b7bf5ce1bfaa2fded 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q
@@ -4,9 +4,9 @@ CREATE DATABASE db2;
 
 CREATE TABLE db1.sourceTable (one string,two string) PARTITIONED BY (ds string);
 
-load data local inpath '../data/files/kv1.txt' INTO TABLE db1.sourceTable partition(ds='2011-11-11');
+load data local inpath '../../data/files/kv1.txt' INTO TABLE db1.sourceTable partition(ds='2011-11-11');
 
-load data local inpath '../data/files/kv3.txt' INTO TABLE db1.sourceTable partition(ds='2011-11-11');
+load data local inpath '../../data/files/kv3.txt' INTO TABLE db1.sourceTable partition(ds='2011-11-11');
 
 CREATE TABLE db2.destinTable (one string,two string) PARTITIONED BY (ds string);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q
index e58b212e190bb7ea1f60eef24215b2ea9125415c..4ff0edcd61bca3b00c7a437a4fdd81df66942e87 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q
@@ -4,10 +4,10 @@ DROP TABLE insert_into3b;
 CREATE TABLE insert_into3a (key int, value string);
 CREATE TABLE insert_into3b (key int, value string);
 
-EXPLAIN FROM src INSERT INTO TABLE insert_into3a SELECT * LIMIT 50
-                 INSERT INTO TABLE insert_into3b SELECT * LIMIT 100;
-FROM src INSERT INTO TABLE insert_into3a SELECT * LIMIT 50
-         INSERT INTO TABLE insert_into3b SELECT * LIMIT 100;
+EXPLAIN FROM src INSERT INTO TABLE insert_into3a SELECT * ORDER BY key, value LIMIT 50
+                 INSERT INTO TABLE insert_into3b SELECT * ORDER BY key, value LIMIT 100;
+FROM src INSERT INTO TABLE insert_into3a SELECT * ORDER BY key, value LIMIT 50
+         INSERT INTO TABLE insert_into3b SELECT * ORDER BY key, value LIMIT 100;
 SELECT SUM(HASH(c)) FROM (
     SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into3a
 ) t;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q
index 25c127f67f98069aed1ed2406df66ca7711d17ca..6d069f5411d4508ac66f698ffe8765525cc4c582 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q
@@ -1,40 +1,40 @@
-insert overwrite local directory '../data/files/local_src_table_1'
+insert overwrite local directory '../../data/files/local_src_table_1'
 select * from src ;
-dfs -cat ../data/files/local_src_table_1/000000_0;
+dfs -cat ../../data/files/local_src_table_1/000000_0;
 
-insert overwrite local directory '../data/files/local_src_table_2'
+insert overwrite local directory '../../data/files/local_src_table_2'
 row format delimited 
 FIELDS TERMINATED BY ':' 
 select * from src ;
 
-dfs -cat ../data/files/local_src_table_2/000000_0;
+dfs -cat ../../data/files/local_src_table_2/000000_0;
 
 create table array_table (a array<string>, b array<string>)
 ROW FORMAT DELIMITED
 FIELDS TERMINATED BY '\t'
 COLLECTION ITEMS TERMINATED BY ',';
 
-load data local inpath "../data/files/array_table.txt" overwrite into table array_table;
+load data local inpath "../../data/files/array_table.txt" overwrite into table array_table;
 
-insert overwrite local directory '../data/files/local_array_table_1'
+insert overwrite local directory '../../data/files/local_array_table_1'
 select * from array_table;
-dfs -cat ../data/files/local_array_table_1/000000_0;
+dfs -cat ../../data/files/local_array_table_1/000000_0;
 
-insert overwrite local directory '../data/files/local_array_table_2'
+insert overwrite local directory '../../data/files/local_array_table_2'
 ROW FORMAT DELIMITED
 FIELDS TERMINATED BY ':'
 COLLECTION ITEMS TERMINATED BY '#'
 select * from array_table;
 
-dfs -cat ../data/files/local_array_table_2/000000_0;
+dfs -cat ../../data/files/local_array_table_2/000000_0;
 
-insert overwrite local directory '../data/files/local_array_table_2_withfields'
+insert overwrite local directory '../../data/files/local_array_table_2_withfields'
 ROW FORMAT DELIMITED
 FIELDS TERMINATED BY ':'
 COLLECTION ITEMS TERMINATED BY '#'
 select b,a from array_table;
 
-dfs -cat ../data/files/local_array_table_2_withfields/000000_0;
+dfs -cat ../../data/files/local_array_table_2_withfields/000000_0;
 
 
 create table map_table (foo STRING , bar MAP<STRING, STRING>)
@@ -44,63 +44,63 @@ COLLECTION ITEMS TERMINATED BY ','
 MAP KEYS TERMINATED BY ':'
 STORED AS TEXTFILE;
 
-load data local inpath "../data/files/map_table.txt" overwrite into table map_table;
+load data local inpath "../../data/files/map_table.txt" overwrite into table map_table;
 
-insert overwrite local directory '../data/files/local_map_table_1'
+insert overwrite local directory '../../data/files/local_map_table_1'
 select * from map_table;
-dfs -cat ../data/files/local_map_table_1/000000_0;
+dfs -cat ../../data/files/local_map_table_1/000000_0;
 
-insert overwrite local directory '../data/files/local_map_table_2'
+insert overwrite local directory '../../data/files/local_map_table_2'
 ROW FORMAT DELIMITED
 FIELDS TERMINATED BY ':'
 COLLECTION ITEMS TERMINATED BY '#'
 MAP KEYS TERMINATED BY '='
 select * from map_table;
 
-dfs -cat ../data/files/local_map_table_2/000000_0;
+dfs -cat ../../data/files/local_map_table_2/000000_0;
 
-insert overwrite local directory '../data/files/local_map_table_2_withfields'
+insert overwrite local directory '../../data/files/local_map_table_2_withfields'
 ROW FORMAT DELIMITED
 FIELDS TERMINATED BY ':'
 COLLECTION ITEMS TERMINATED BY '#'
 MAP KEYS TERMINATED BY '='
 select bar,foo from map_table;
 
-dfs -cat ../data/files/local_map_table_2_withfields/000000_0;
+dfs -cat ../../data/files/local_map_table_2_withfields/000000_0;
 
-insert overwrite local directory '../data/files/local_array_table_3'
+insert overwrite local directory '../../data/files/local_array_table_3'
 ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.DelimitedJSONSerDe'
 STORED AS TEXTFILE
 select * from array_table;
 
-dfs -cat ../data/files/local_array_table_3/000000_0;
+dfs -cat ../../data/files/local_array_table_3/000000_0;
 
-insert overwrite local directory '../data/files/local_map_table_3'
+insert overwrite local directory '../../data/files/local_map_table_3'
 ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.DelimitedJSONSerDe'
 STORED AS TEXTFILE
 select * from map_table;
 
-dfs -cat ../data/files/local_map_table_3/000000_0;
+dfs -cat ../../data/files/local_map_table_3/000000_0;
 
-insert overwrite local directory '../data/files/local_rctable'
+insert overwrite local directory '../../data/files/local_rctable'
 STORED AS RCFILE
 select value,key from src;
 
 dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/local_rctable/temp;
 dfs -rmr ${system:test.tmp.dir}/local_rctable;
 dfs ${system:test.dfs.mkdir}  ${system:test.tmp.dir}/local_rctable;
-dfs -put ../data/files/local_rctable/000000_0 ${system:test.tmp.dir}/local_rctable/000000_0;
+dfs -put ../../data/files/local_rctable/000000_0 ${system:test.tmp.dir}/local_rctable/000000_0;
 
 create external table local_rctable(value string, key string)
 STORED AS RCFILE
 LOCATION '${system:test.tmp.dir}/local_rctable';
 
-insert overwrite local directory '../data/files/local_rctable_out'
+insert overwrite local directory '../../data/files/local_rctable_out'
 ROW FORMAT DELIMITED
 FIELDS TERMINATED BY '\t'
 select key,value from local_rctable;
 
-dfs -cat ../data/files/local_rctable_out/000000_0;
+dfs -cat ../../data/files/local_rctable_out/000000_0;
 
 drop table local_rctable;
 drop table array_table;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q
index b403814adb4c9b1d4e76040c078dd407d97a7750..4d1ae2186e796cd49c39e00e0f924f58c5a9c72c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q
@@ -1,9 +1,9 @@
 
 CREATE TABLE join_1to1_1(key1 int, key2 int, value int);
-LOAD DATA LOCAL INPATH '../data/files/in5.txt' INTO TABLE join_1to1_1;
+LOAD DATA LOCAL INPATH '../../data/files/in5.txt' INTO TABLE join_1to1_1;
 
 CREATE TABLE join_1to1_2(key1 int, key2 int, value int);
-LOAD DATA LOCAL INPATH '../data/files/in6.txt' INTO TABLE join_1to1_2;
+LOAD DATA LOCAL INPATH '../../data/files/in6.txt' INTO TABLE join_1to1_2;
 
 
 set hive.outerjoin.supports.filters=false;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_alt_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_alt_syntax.q
new file mode 100644
index 0000000000000000000000000000000000000000..0b0c53803d01e4ce7668c46e55374e65c22bf1ff
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_alt_syntax.q
@@ -0,0 +1,41 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+explain select p1.p_name, p2.p_name
+from part p1 , part p2;
+
+explain select p1.p_name, p2.p_name, p3.p_name
+from part p1 ,part p2 ,part p3 
+where p1.p_name = p2.p_name and p2.p_name = p3.p_name;
+
+explain select p1.p_name, p2.p_name, p3.p_name
+from part p1 , (select p_name from part) p2 ,part p3 
+where p1.p_name = p2.p_name and p2.p_name = p3.p_name;
+
+explain select p1.p_name, p2.p_name, p3.p_name
+from part p1 , part p2 , part p3 
+where p2.p_partkey + p1.p_partkey = p1.p_partkey and p3.p_name = p2.p_name;
+
+explain select p1.p_name, p2.p_name, p3.p_name, p4.p_name
+from part p1 , part p2 join part p3 on p2.p_name = p1.p_name join part p4 
+where p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey 
+            and p1.p_partkey = p2.p_partkey;
+            
+explain select p1.p_name, p2.p_name, p3.p_name, p4.p_name
+from part p1 join part p2 on p2.p_name = p1.p_name , part p3  , part p4 
+where p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey 
+            and p1.p_partkey = p2.p_partkey;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q
index e4d95a51fa927f9c7003572e95104d4836315bf6..81e984e7eb01867cfdd4229621c5475b3ef8656e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q
@@ -1,8 +1,8 @@
 create table tinyA(a bigint, b bigint) stored as textfile;
 create table tinyB(a bigint, bList array<int>) stored as textfile;
 
-load data local inpath '../data/files/tiny_a.txt' into table tinyA;
-load data local inpath '../data/files/tiny_b.txt' into table tinyB;
+load data local inpath '../../data/files/tiny_a.txt' into table tinyA;
+load data local inpath '../../data/files/tiny_b.txt' into table tinyB;
 
 select * from tinyA;
 select * from tinyB;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q
index a9b69c5b1c44822b04a55f2851e375199c3ee4d9..0c0962ceceebcd1e402b4b6243f0bfa64325c87a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q
@@ -1,8 +1,8 @@
 
 CREATE TABLE joinone(key1 int, key2 int, value int);
-LOAD DATA LOCAL INPATH '../data/files/in5.txt' INTO TABLE joinone;
+LOAD DATA LOCAL INPATH '../../data/files/in5.txt' INTO TABLE joinone;
 
 CREATE TABLE joinTwo(key1 int, key2 int, value int);
-LOAD DATA LOCAL INPATH '../data/files/in6.txt' INTO TABLE joinTwo;
+LOAD DATA LOCAL INPATH '../../data/files/in6.txt' INTO TABLE joinTwo;
 
 SELECT * FROM joinone JOIN joinTwo ON(joinone.key2=joinTwo.key2) ORDER BY joinone.key1 ASC, joinone.key2 ASC, joinone.value ASC, joinTwo.key1 ASC, joinTwo.key2 ASC, joinTwo.value ASC;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_1.q
new file mode 100644
index 0000000000000000000000000000000000000000..7f493671b80c163fd481176e60b70b90ff54fe1d
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_1.q
@@ -0,0 +1,30 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+
+
+explain select *
+from part p1 join part p2 join part p3 on p1.p_name = p2.p_name and p2.p_name = p3.p_name;
+
+explain select *
+from part p1 join part p2 join part p3 on p2.p_name = p1.p_name and p3.p_name = p2.p_name;
+
+explain select *
+from part p1 join part p2 join part p3 on p2.p_partkey + p1.p_partkey = p1.p_partkey and p3.p_name = p2.p_name;
+
+explain select *
+from part p1 join part p2 join part p3 on p2.p_partkey = 1 and p3.p_name = p2.p_name;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_2.q
new file mode 100644
index 0000000000000000000000000000000000000000..ca280104d9a75cad2f90866fc590af2ac8f9448e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_2.q
@@ -0,0 +1,24 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+
+explain select *
+from part p1 join part p2 join part p3 on p1.p_name = p2.p_name join part p4 on p2.p_name = p3.p_name and p1.p_name = p4.p_name;
+
+explain select *
+from part p1 join part p2 join part p3 on p2.p_name = p1.p_name join part p4 on p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey 
+            and p1.p_partkey = p2.p_partkey;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_3.q
new file mode 100644
index 0000000000000000000000000000000000000000..b308838d6243b392b744a21246306c3626e0bfca
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_3.q
@@ -0,0 +1,34 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+
+
+explain select *
+from part p1 join part p2 join part p3 
+where p1.p_name = p2.p_name and p2.p_name = p3.p_name;
+
+explain select *
+from part p1 join part p2 join part p3 
+where p2.p_name = p1.p_name and p3.p_name = p2.p_name;
+
+explain select *
+from part p1 join part p2 join part p3 
+where p2.p_partkey + p1.p_partkey = p1.p_partkey and p3.p_name = p2.p_name;
+
+explain select *
+from part p1 join part p2 join part p3 
+where p2.p_partkey = 1 and p3.p_name = p2.p_name;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_4.q
new file mode 100644
index 0000000000000000000000000000000000000000..477682e4e8ac7e237975d9f2afff02af6848c186
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_4.q
@@ -0,0 +1,26 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+
+explain select *
+from part p1 join part p2 join part p3 on p1.p_name = p2.p_name join part p4 
+where p2.p_name = p3.p_name and p1.p_name = p4.p_name;
+
+explain select *
+from part p1 join part p2 join part p3 on p2.p_name = p1.p_name join part p4 
+where p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey 
+            and p1.p_partkey = p2.p_partkey;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual1.q
new file mode 100644
index 0000000000000000000000000000000000000000..1013f51a0d806266c98cf9f672c2a04eb04def10
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual1.q
@@ -0,0 +1,52 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+create table part2( 
+    p2_partkey INT,
+    p2_name STRING,
+    p2_mfgr STRING,
+    p2_brand STRING,
+    p2_type STRING,
+    p2_size INT,
+    p2_container STRING,
+    p2_retailprice DOUBLE,
+    p2_comment STRING
+);
+
+create table part3( 
+    p3_partkey INT,
+    p3_name STRING,
+    p3_mfgr STRING,
+    p3_brand STRING,
+    p3_type STRING,
+    p3_size INT,
+    p3_container STRING,
+    p3_retailprice DOUBLE,
+    p3_comment STRING
+);
+
+explain select *
+from part p1 join part2 p2 join part3 p3 on p1.p_name = p2_name and p2_name = p3_name;
+
+explain select *
+from part p1 join part2 p2 join part3 p3 on p2_name = p1.p_name and p3_name = p2_name;
+
+explain select *
+from part p1 join part2 p2 join part3 p3 on p2_partkey + p_partkey = p1.p_partkey and p3_name = p2_name;
+
+explain select *
+from part p1 join part2 p2 join part3 p3 on p2_partkey = 1 and p3_name = p2_name;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual2.q
new file mode 100644
index 0000000000000000000000000000000000000000..62323577528516cf193b1ce67d57c3565201bbe2
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual2.q
@@ -0,0 +1,47 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+create table part2( 
+    p2_partkey INT,
+    p2_name STRING,
+    p2_mfgr STRING,
+    p2_brand STRING,
+    p2_type STRING,
+    p2_size INT,
+    p2_container STRING,
+    p2_retailprice DOUBLE,
+    p2_comment STRING
+);
+
+create table part3( 
+    p3_partkey INT,
+    p3_name STRING,
+    p3_mfgr STRING,
+    p3_brand STRING,
+    p3_type STRING,
+    p3_size INT,
+    p3_container STRING,
+    p3_retailprice DOUBLE,
+    p3_comment STRING
+);
+
+explain select *
+from part p1 join part2 p2 join part3 p3 on p1.p_name = p2_name join part p4 on p2_name = p3_name and p1.p_name = p4.p_name;
+
+explain select *
+from part p1 join part2 p2 join part3 p3 on p2_name = p1.p_name join part p4 on p2_name = p3_name and p1.p_partkey = p4.p_partkey 
+            and p1.p_partkey = p2_partkey;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual3.q
new file mode 100644
index 0000000000000000000000000000000000000000..6ac86042c04526be9f911eb9002b947418a85b8a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual3.q
@@ -0,0 +1,56 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+create table part2( 
+    p2_partkey INT,
+    p2_name STRING,
+    p2_mfgr STRING,
+    p2_brand STRING,
+    p2_type STRING,
+    p2_size INT,
+    p2_container STRING,
+    p2_retailprice DOUBLE,
+    p2_comment STRING
+);
+
+create table part3( 
+    p3_partkey INT,
+    p3_name STRING,
+    p3_mfgr STRING,
+    p3_brand STRING,
+    p3_type STRING,
+    p3_size INT,
+    p3_container STRING,
+    p3_retailprice DOUBLE,
+    p3_comment STRING
+);
+
+explain select *
+from part p1 join part2 p2 join part3 p3 
+where p1.p_name = p2_name and p2_name = p3_name;
+
+explain select *
+from part p1 join part2 p2 join part3 p3 
+where p2_name = p1.p_name and p3_name = p2_name;
+
+explain select *
+from part p1 join part2 p2 join part3 p3 
+where p2_partkey + p1.p_partkey = p1.p_partkey and p3_name = p2_name;
+
+explain select *
+from part p1 join part2 p2 join part3 p3 
+where p2_partkey = 1 and p3_name = p2_name;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual4.q
new file mode 100644
index 0000000000000000000000000000000000000000..0db4d5e3cff1ae8bba276d9ad5eba8a28760452d
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual4.q
@@ -0,0 +1,49 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+create table part2( 
+    p2_partkey INT,
+    p2_name STRING,
+    p2_mfgr STRING,
+    p2_brand STRING,
+    p2_type STRING,
+    p2_size INT,
+    p2_container STRING,
+    p2_retailprice DOUBLE,
+    p2_comment STRING
+);
+
+create table part3( 
+    p3_partkey INT,
+    p3_name STRING,
+    p3_mfgr STRING,
+    p3_brand STRING,
+    p3_type STRING,
+    p3_size INT,
+    p3_container STRING,
+    p3_retailprice DOUBLE,
+    p3_comment STRING
+);
+
+explain select *
+from part p1 join part2 p2 join part3 p3 on p1.p_name = p2_name join part p4 
+where p2_name = p3_name and p1.p_name = p4.p_name;
+
+explain select *
+from part p1 join part2 p2 join part3 p3 on p2_name = p1.p_name join part p4 
+where p2_name = p3_name and p1.p_partkey = p4.p_partkey 
+            and p1.p_partkey = p2_partkey;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q
index d54aa950a1ec22e57f0000c9b39e0445805b6aea..49b6c6f920209bf244aedd44839e2d5abb5d0010 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q
@@ -1,5 +1,5 @@
 CREATE TABLE myinput1(key int, value int);
-LOAD DATA LOCAL INPATH '../data/files/in3.txt' INTO TABLE myinput1;
+LOAD DATA LOCAL INPATH '../../data/files/in3.txt' INTO TABLE myinput1;
 
 SELECT * FROM myinput1 a JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC;
 SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC;
@@ -55,10 +55,10 @@ SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.valu
 
 CREATE TABLE smb_input1(key int, value int) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; 
 CREATE TABLE smb_input2(key int, value int) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS; 
-LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input1;
-LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input1;
-LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input2;
-LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input2;
+LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input1;
+LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input1;
+LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input2;
+LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input2;
 
 SET hive.optimize.bucketmapjoin = true;
 SET hive.optimize.bucketmapjoin.sortedmerge = true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q
index 31b0c8c91c10b0c21a2770f0fac9f80e47a09902..c4c239cae2d8ac99723a15d17942497f35b36d2d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q
@@ -13,9 +13,9 @@ delimited fields terminated by ',' stored as textfile;
 create table hive_count (bar_id int, n int) row format delimited fields 
 terminated by ',' stored as textfile;
 
-load data local inpath '../data/files/hive_626_foo.txt' overwrite into table hive_foo;
-load data local inpath '../data/files/hive_626_bar.txt' overwrite into table hive_bar;
-load data local inpath '../data/files/hive_626_count.txt' overwrite into table hive_count;
+load data local inpath '../../data/files/hive_626_foo.txt' overwrite into table hive_foo;
+load data local inpath '../../data/files/hive_626_bar.txt' overwrite into table hive_bar;
+load data local inpath '../../data/files/hive_626_count.txt' overwrite into table hive_count;
 
 explain
 select hive_foo.foo_name, hive_bar.bar_name, n from hive_foo join hive_bar on hive_foo.foo_id =
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_merging.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_merging.q
new file mode 100644
index 0000000000000000000000000000000000000000..a0046dbc413325304cac246b9d402fca47d50b9d
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_merging.q
@@ -0,0 +1,25 @@
+
+
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+explain select p1.p_size, p2.p_size 
+from part p1 left outer join part p2 on p1.p_partkey = p2.p_partkey 
+  right outer join part p3 on p2.p_partkey = p3.p_partkey and 
+              p1.p_size > 10
+;
+
+explain select p1.p_size, p2.p_size 
+from part p1 left outer join part p2 on p1.p_partkey = p2.p_partkey 
+  right outer join part p3 on p2.p_partkey = p3.p_partkey and 
+              p1.p_size > 10 and p1.p_size > p2.p_size + 10
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q
index 4ff60713d6b443dc4a0234ce3f45338398cd1062..047a769eb8e95eac70b771b39ae0fe0ba2fa28bb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q
@@ -1,5 +1,5 @@
 CREATE TABLE myinput1(key int, value int);
-LOAD DATA LOCAL INPATH '../data/files/in1.txt' INTO TABLE myinput1;
+LOAD DATA LOCAL INPATH '../../data/files/in1.txt' INTO TABLE myinput1;
 
 SELECT * FROM myinput1 a JOIN myinput1 b ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC;
 SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC;
@@ -42,10 +42,10 @@ SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.valu
 
 CREATE TABLE smb_input1(key int, value int) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; 
 CREATE TABLE smb_input2(key int, value int) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS; 
-LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input1;
-LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input1;
-LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input2;
-LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input2;
+LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input1;
+LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input1;
+LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input2;
+LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input2;
 
 SET hive.optimize.bucketmapJOIN = true;
 SET hive.optimize.bucketmapJOIN.sortedmerge = true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q
index 05b57bce202d28b05417f4ca57a997f0eafa0d99..5e22517edbd7270d71a66e22de517b73a23c26b0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q
@@ -1,7 +1,7 @@
 set hive.nullsafe.equijoin=true;
 
 CREATE TABLE myinput1(key int, value int);
-LOAD DATA LOCAL INPATH '../data/files/in8.txt' INTO TABLE myinput1;
+LOAD DATA LOCAL INPATH '../../data/files/in8.txt' INTO TABLE myinput1;
 
 -- merging
 explain select * from myinput1 a join myinput1 b on a.key<=>b.value ORDER BY a.key, a.value, b.key, b.value;
@@ -31,10 +31,10 @@ SELECT /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.key<=>b.value OR
 -- smbs
 CREATE TABLE smb_input1(key int, value int) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS;
 CREATE TABLE smb_input2(key int, value int) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS;
-LOAD DATA LOCAL INPATH '../data/files/in8.txt' into table smb_input1;
-LOAD DATA LOCAL INPATH '../data/files/in9.txt' into table smb_input1;
-LOAD DATA LOCAL INPATH '../data/files/in8.txt' into table smb_input2;
-LOAD DATA LOCAL INPATH '../data/files/in9.txt' into table smb_input2;
+LOAD DATA LOCAL INPATH '../../data/files/in8.txt' into table smb_input1;
+LOAD DATA LOCAL INPATH '../../data/files/in9.txt' into table smb_input1;
+LOAD DATA LOCAL INPATH '../../data/files/in8.txt' into table smb_input2;
+LOAD DATA LOCAL INPATH '../../data/files/in9.txt' into table smb_input2;
 
 SET hive.optimize.bucketmapJOIN = true;
 SET hive.optimize.bucketmapJOIN.sortedmerge = true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q
index b92a79ba07ab292c384a40591b788db6d649fd60..b209c50b66194b40dc183f41301765a2fa47f0f0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q
@@ -6,9 +6,9 @@ CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE;
 CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE;
 CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
-LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3;
 
 EXPLAIN FROM T1 a JOIN src c ON c.key+1=a.key
 SELECT a.key, a.val, c.key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q
index 238c0adad3128559dcdcec38e18aa705d9d2b114..ca1e65ebef6f018d362e6994bdb2412e3ff05733 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q
@@ -8,10 +8,10 @@ CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE;
 CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE;
 CREATE TABLE T4(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
-LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3;
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T4;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T4;
 
 EXPLAIN
 SELECT /*+ STREAMTABLE(a) */ *
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q
index 1bda28fbc3d5f8eae69ab1c35b8ea89811748a2e..994be164aa62c0b74cb6bbd8dfb8cfcaaa129e50 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q
@@ -8,10 +8,10 @@ CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE;
 CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE;
 CREATE TABLE T4(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
-LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3;
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T4;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T4;
 
 EXPLAIN
 SELECT /*+ STREAMTABLE(a,c) */ *
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q
index 126f356ef785ad9e7de778ee8de8b71c0826b87b..16ef2046be35a22f7554fa454f85c182c53bbae4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q
@@ -2,9 +2,9 @@ CREATE TABLE T1(key1 STRING, val1 STRING) STORED AS TEXTFILE;
 CREATE TABLE T2(key2 STRING, val2 STRING) STORED AS TEXTFILE;
 CREATE TABLE T3(key3 STRING, val3 STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
-LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3;
 
 set hive.auto.convert.join=true;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q
index 8314161975761415c3d6cb5f6a1ad9d57445a3d5..c95a13b9cac5aaef296c9568e22470e9e5161571 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q
@@ -7,14 +7,14 @@ create table dim5(f9 int, f10 int);
 create table dim6(f11 int, f12 int);
 create table dim7(f13 int, f14 int);
 
-LOAD DATA LOCAL INPATH '../data/files/fact-data.txt' INTO TABLE fact;
-LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim1;
-LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim2;
-LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim3;
-LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim4;
-LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim5;
-LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim6;
-LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim7;
+LOAD DATA LOCAL INPATH '../../data/files/fact-data.txt' INTO TABLE fact;
+LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim1;
+LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim2;
+LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim3;
+LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim4;
+LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim5;
+LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim6;
+LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim7;
 
 set hive.auto.convert.join=true;
 set hive.auto.convert.join.noconditionaltask=true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q
index 11e54a5b04c07f2aed859e114e914d49cb49837c..df7343259ae75fcfc26096b88f197737a53df62f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 --HIVE-2608 Do not require AS a,b,c part in LATERAL VIEW
 EXPLAIN SELECT myTab.* from src LATERAL VIEW explode(map('key1', 100, 'key2', 200)) myTab limit 2;
 SELECT myTab.* from src LATERAL VIEW explode(map('key1', 100, 'key2', 200)) myTab limit 2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q
index 7be86a6f10ea2b794ffddd3f06430fafaaa3f037..65ae518cd5be2f35359c74bfde9ae1c8a06e2f6e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q
@@ -11,3 +11,7 @@ SELECT value, myCol FROM (SELECT * FROM srcpart LATERAL VIEW explode(array(1,2,3
 
 EXPLAIN SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol LATERAL VIEW explode(array(1,2,3)) myTable2 AS myCol2) a WHERE key='0';
 SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol LATERAL VIEW explode(array(1,2,3)) myTable2 AS myCol2) a WHERE key='0';
+
+-- HIVE-4293 Predicates following UDTF operator are removed by PPD
+EXPLAIN SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol WHERE myCol > 1) a WHERE key='0';
+SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol WHERE myCol > 1) a WHERE key='0';
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lb_fs_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lb_fs_stats.q
new file mode 100644
index 0000000000000000000000000000000000000000..7f31797f314b3b14f13b507919c0840534d4a291
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lb_fs_stats.q
@@ -0,0 +1,19 @@
+set hive.mapred.supports.subdirectories=true;
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+set hive.merge.mapfiles=false;	
+set hive.merge.mapredfiles=false; 
+set mapred.input.dir.recursive=true;
+set hive.stats.dbclass=fs;
+-- Tests truncating a column from a list bucketing table
+
+-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23)
+
+CREATE TABLE test_tab (key STRING, value STRING) PARTITIONED BY (part STRING) STORED AS RCFILE;
+
+ALTER TABLE test_tab SKEWED BY (key) ON ("484") STORED AS DIRECTORIES;
+
+INSERT OVERWRITE TABLE test_tab PARTITION (part = '1') SELECT * FROM src;
+
+describe formatted test_tab partition (part='1');
+
+set hive.stats.dbclass=jdbc:derby;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q
index f49766771067bb42de2f32db01b6d5894e4b4d35..5623cbfac51a7c7dcb3cabf6a7393c6d6b7a890a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q
@@ -13,7 +13,7 @@ CREATE TABLE part(
     p_comment STRING
 );
 
-LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part;
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
 
 --1. testLagWithPTFWindowing
 select p_mfgr, p_name,
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q
index 6ef3bdb107a1229305453ade309a546242a6871c..e53abce763865f99662710517db657a327f49cad 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q
@@ -11,7 +11,7 @@ CREATE TABLE part(
     p_comment STRING
 );
 
-LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part;
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
 
 -- 1. testLeadUDAF
 select p_mfgr, p_retailprice,
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q
index abe3d3317157c9e3380fffb854b02495eb163683..0c16fb8dfd9887af1bf55385b206da0e87952f44 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q
@@ -9,9 +9,9 @@ ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t';
 CREATE TABLE things (id INT, name STRING) partitioned by (ds string)
 ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t';
 
-load data local inpath '../data/files/sales.txt' INTO TABLE sales;
-load data local inpath '../data/files/things.txt' INTO TABLE things partition(ds='2011-10-23');
-load data local inpath '../data/files/things2.txt' INTO TABLE things partition(ds='2011-10-24');
+load data local inpath '../../data/files/sales.txt' INTO TABLE sales;
+load data local inpath '../../data/files/things.txt' INTO TABLE things partition(ds='2011-10-23');
+load data local inpath '../../data/files/things2.txt' INTO TABLE things partition(ds='2011-10-24');
 
 SELECT name,id FROM sales ORDER BY name ASC, id ASC;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q
index 5813ca3c619cc8c39d0453ab1d53eba3b430496b..c9ebe0e8fad127b15b1890a4ae75e55d7de18333 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q
@@ -1,7 +1,7 @@
 CREATE TABLE T1(key INT);
-LOAD DATA LOCAL INPATH '../data/files/leftsemijoin_mr_t1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/leftsemijoin_mr_t1.txt' INTO TABLE T1;
 CREATE TABLE T2(key INT);
-LOAD DATA LOCAL INPATH '../data/files/leftsemijoin_mr_t2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/leftsemijoin_mr_t2.txt' INTO TABLE T2;
 
 -- Run this query using TestMinimrCliDriver
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q
new file mode 100644
index 0000000000000000000000000000000000000000..e91adab59ddfa631e5c6dab693c35be534ccc49b
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q
@@ -0,0 +1,7 @@
+set hive.limit.query.max.table.partition=1;
+
+explain select ds from srcpart where hr=11 and ds='2008-04-08';
+select ds from srcpart where hr=11 and ds='2008-04-08';
+
+explain select distinct hr from srcpart;
+select distinct hr from srcpart;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q
index e4d0aa06bde871242165cad8a44aef5ebac1aca4..adfe1e63e889f8c390f76dc6ac86bb7732bc548d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q
@@ -22,12 +22,17 @@ select value,avg(key + 1) from src group by value order by value limit 20;
 
 -- distincts
 explain
-select distinct(key) from src limit 20;
-select distinct(key) from src limit 20;
+select distinct(cdouble) from alltypesorc limit 20;
+select distinct(cdouble) from alltypesorc limit 20;
 
 explain
-select key, count(distinct(key)) from src group by key limit 20;
-select key, count(distinct(key)) from src group by key limit 20;
+select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint limit 20;
+select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint limit 20;
+
+-- multi distinct
+explain
+select ctinyint, count(distinct(cstring1)), count(distinct(cstring2)) from alltypesorc group by ctinyint limit 20;
+select ctinyint, count(distinct(cstring1)), count(distinct(cstring2)) from alltypesorc group by ctinyint limit 20;
 
 -- limit zero
 explain
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q
index a86ddf14047efd7dab2e24ee59ca9bcc05c9bbee..e17ded1ee1b9b6251c29039c2adc868c1f6fa8a2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q
@@ -16,7 +16,3 @@ CREATE TABLE dest_3(key STRING, c1 INT);
 EXPLAIN FROM src
 INSERT OVERWRITE TABLE dest_2 SELECT value, sum(key) GROUP BY value
 INSERT OVERWRITE TABLE dest_3 SELECT value, sum(key) GROUP BY value limit 20;
-
--- nagative, multi distinct
-explain
-select count(distinct key)+count(distinct value) from src limit 20;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q
index 3a39f42b2a77df8cd3b4e4fa6ae93c3fb1285ae0..80aba5d4a526f925aafd0d9a190a6ff5d7a96289 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q
@@ -48,13 +48,13 @@ set hive.optimize.listbucketing=true;
 explain extended
 select * from list_bucketing_static_part where ds = '2008-04-08' and  hr = '11' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr;
 select * from list_bucketing_static_part where ds = '2008-04-08' and  hr = '11' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr;
-select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' ORDER BY key, value;
+select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr;
 
 -- 51 and val_51 in the table so skewed data for 51 and val_14 should be none
 -- but query should succeed for 51 or 51 and val_14
-select * from srcpart where ds = '2008-04-08' and key = '51' ORDER BY key, value;
+select * from srcpart where ds = '2008-04-08' and key = '51' ORDER BY key, value, ds, hr;
 select * from list_bucketing_static_part where key = '51' ORDER BY key, value, ds, hr;
-select * from srcpart where ds = '2008-04-08' and key = '51' and value = 'val_14'  ORDER BY key, value;
+select * from srcpart where ds = '2008-04-08' and key = '51' and value = 'val_14'  ORDER BY key, value, ds, hr;
 select * from list_bucketing_static_part where key = '51' and value = 'val_14' ORDER BY key, value, ds, hr;
 
 -- queries with < <= > >= should work for skewed test although we don't benefit from pruning
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q
index 918c817e49eff481337ba2b77c739e49bddd7a36..380d148ac9f4392cfea7d829c7da9f6daa6b7111 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q
@@ -65,7 +65,7 @@ set hive.optimize.listbucketing=true;
 explain extended
 select * from list_bucketing_static_part where ds = '2008-04-08' and  hr = '11' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr;
 select * from list_bucketing_static_part where ds = '2008-04-08' and  hr = '11' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr;
-select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' ORDER BY key, value;
+select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr;
 
 -- clean up
 drop table list_bucketing_static_part;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q
index a6ad4b8485a26ec1309127e619ccc447010b1e3f..08b21dc68942436fa31a48274ad9f410fdd8b11e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 EXPLAIN SELECT -1BD, 0BD, 1BD, 3.14BD, -3.14BD, 99999999999999999BD, 99999999999999999.9999999999999BD, 1E-99BD, 1E99BD FROM src LIMIT 1;
 
 SELECT -1BD, 0BD, 1BD, 3.14BD, -3.14BD, 99999999999999999BD, 99999999999999999.9999999999999BD, 1E-99BD, 1E99BD FROM src LIMIT 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q
index 08836127b99584d9ce07004841687e3d0de60314..766da699ea2508da8f85b3e8525c97c917df16ab 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 EXPLAIN SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 FROM src LIMIT 1;
 SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 FROM src LIMIT 1;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q
index 9da622e246793a12afafe83bbcd825dbd73ce878..5fd0cfabf3076b1ff23173530e1baa08d6b2d452 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 EXPLAIN SELECT 100, 100Y, 100S, 100L FROM src LIMIT 1;
 
 SELECT 100, 100Y, 100S, 100L FROM src LIMIT 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q
index 21f0890ada168331ca57c69766293e0d2c1cefa1..c57dc572529e17688e09f48ecc77cd36815c9a69 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 EXPLAIN SELECT 'face''book', 'face' 'book', 'face'
                                             'book',
                "face""book", "face" "book", "face"
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q
index 7da363183ccf660417711ffd8c267e03b7c614c1..653918afc0fa3fe98d1dedc3de59dd26a812ddf8 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q
@@ -4,7 +4,7 @@ FIELDS TERMINATED BY '9'
 STORED AS TEXTFILE;
 -- this query loads native binary data, stores in a table and then queries it. Note that string.txt contains binary data. Also uses transform clause and then length udf.
 
-LOAD DATA LOCAL INPATH '../data/files/string.txt' INTO TABLE mytable;
+LOAD DATA LOCAL INPATH '../../data/files/string.txt' INTO TABLE mytable;
 
 create table dest1 (key binary, value int);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q
index 52b4937d4a9dd09009b27793b7d74bd0fb42fb84..5f0a015693d9fede403362b365f6604b70e1efe2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q
@@ -23,8 +23,8 @@ insert overwrite table nzhang_part2 partition(ds='2008-12-31', hr) select key, v
 show partitions nzhang_part1;
 show partitions nzhang_part2;
 
-select * from nzhang_part1 where ds is not null and hr is not null;
-select * from nzhang_part2 where ds is not null and hr is not null;
+select * from nzhang_part1 where ds is not null and hr is not null order by ds, hr, key;
+select * from nzhang_part2 where ds is not null and hr is not null order by ds, hr, key;
 
 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q
index 9517664675d694e3b55aeb2dd9d07ba3a3cf75ee..dd84599e69e6d89e38e697e90400e678a53c7cac 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q
@@ -19,6 +19,6 @@ insert overwrite table nzhang_part10 partition(ds='2008-12-31', hr) select key,
 
 show partitions nzhang_part10;
 
-select * from nzhang_part10 where ds is not null and hr is not null;
+select * from nzhang_part10 where ds is not null and hr is not null order by ds, hr, key;
 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q
index e4c8c17f63048b4bc762b42477a48dabd579bda0..29f951aa69f67304803d984bacaabfec2dc2f4e5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q
@@ -14,6 +14,6 @@ insert overwrite table nzhang_part3 partition (ds, hr) select key, value, ds, hr
 
 insert overwrite table nzhang_part3 partition (ds, hr) select key, value, ds, hr from srcpart where ds is not null and hr is not null;
 
-select * from nzhang_part3 where ds is not null and hr is not null;
+select * from nzhang_part3 where ds is not null and hr is not null order by ds, hr, key;
 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q
index 3f3a0c8d51b4189444260908a0f1b41ac4c3454d..942c245db8b6481be7f6c78c65d405cb45bbac91 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q
@@ -17,8 +17,8 @@ insert overwrite table nzhang_part4 partition (ds, hr) select key, value, ds, hr
 insert overwrite table nzhang_part4 partition (ds, hr) select key, value, ds, hr from srcpart where ds is not null and hr is not null;
 
 show partitions nzhang_part4;
-select * from nzhang_part4 where ds='2008-04-08' and hr is not null;
+select * from nzhang_part4 where ds='2008-04-08' and hr is not null order by ds, hr, key;
 
-select * from nzhang_part4 where ds is not null and hr is not null;
+select * from nzhang_part4 where ds is not null and hr is not null order by ds, hr, key;
 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q
index 8073500c0bf07f328d9032c05108aef3338696cc..6768e4373a05664dde9a251d5f57d821ea8c7066 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q
@@ -20,5 +20,5 @@ insert overwrite table nzhang_part8 partition(ds='2008-12-31', hr) select key, v
 
 show partitions nzhang_part8;
 
-select * from nzhang_part8 where ds is not null and hr is not null;
+select * from nzhang_part8 where ds is not null and hr is not null order by ds, hr, key;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q
index 01fa596cdf04a51c2bb50c09c557566bc99baf2d..4680033cbd541b0e2276911166ea2c08ae2f33da 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q
@@ -19,5 +19,5 @@ insert overwrite table nzhang_part9 partition (ds, hr) select key, value, ds, hr
 
 show partitions nzhang_part9;
 
-select * from nzhang_part9 where ds is not null and hr is not null;
+select * from nzhang_part9 where ds is not null and hr is not null order by ds, hr, key;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q
index 6d2a8b82d33c1eade1c87ecd6b93a25dc695b010..35eb2198081f09b4c87c0f12cd722bfd00917968 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q
@@ -2,4 +2,4 @@ create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored
 alter table hive_test_src add partition (pcol1 = 'test_part');
 set hive.security.authorization.enabled=true;
 grant Update on table hive_test_src to user hive_test_user;
-load data local inpath '../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part');
+load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part');
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q
index 3b8951a1a782f8c3d31f25cdc8e08c10cee4b143..6bac47fb9052acbfb8d2bbb44b80a08fe1293dca 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q
@@ -2,4 +2,5 @@
 
 
 CREATE TABLE load_file_with_space_in_the_name(name STRING, age INT);
-LOAD DATA LOCAL INPATH '../data/files/person age.txt' INTO TABLE load_file_with_space_in_the_name;
+LOAD DATA LOCAL INPATH '../../data/files/person age.txt' INTO TABLE load_file_with_space_in_the_name;
+LOAD DATA LOCAL INPATH '../../data/files/person+age.txt' INTO TABLE load_file_with_space_in_the_name;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q
index c1ac29c172f60e51b6a42f0283c8794b2b7c6f7e..2f06ca464ff9916d41a634f3f20aab5fcefdeaf0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q
@@ -2,9 +2,9 @@
 create table load_overwrite (key string, value string) stored as textfile location 'file:${system:test.tmp.dir}/load_overwrite';
 create table load_overwrite2 (key string, value string) stored as textfile location 'file:${system:test.tmp.dir}/load2_overwrite2';
 
-load data local inpath '../data/files/kv1.txt' into table load_overwrite;
-load data local inpath '../data/files/kv2.txt' into table load_overwrite;
-load data local inpath '../data/files/kv3.txt' into table load_overwrite;
+load data local inpath '../../data/files/kv1.txt' into table load_overwrite;
+load data local inpath '../../data/files/kv2.txt' into table load_overwrite;
+load data local inpath '../../data/files/kv3.txt' into table load_overwrite;
 
 show table extended like load_overwrite;
 desc extended load_overwrite;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q
index 7255324d1653d8e5a7b5988c069e72a1629484d8..a75758a0728d51dac0aac7f4dedbea5733200e7e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q
@@ -4,17 +4,17 @@
 create table result (key string, value string);
 create table loader (key string, value string);
 
-load data local inpath '../data/files/kv1.txt' into table loader;
+load data local inpath '../../data/files/kv1.txt' into table loader;
 
 load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result;
 show table extended like result;
 
-load data local inpath '../data/files/kv1.txt' into table loader;
+load data local inpath '../../data/files/kv1.txt' into table loader;
 
 load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result;
 show table extended like result;
 
-load data local inpath '../data/files/kv1.txt' into table loader;
+load data local inpath '../../data/files/kv1.txt' into table loader;
 
 load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result;
 show table extended like result;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs_overwrite.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs_overwrite.q
new file mode 100644
index 0000000000000000000000000000000000000000..51a803130a660ccd9f84086a8ce75264bb7a8029
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs_overwrite.q
@@ -0,0 +1,20 @@
+--HIVE 6209
+
+drop table target;
+drop table temp;
+
+create table target (key string, value string) stored as textfile location 'file:${system:test.tmp.dir}/target';
+create table temp (key string, value string) stored as textfile location 'file:${system:test.tmp.dir}/temp';
+
+set fs.pfile.impl.disable.cache=false;
+
+load data local inpath '../../data/files/kv1.txt' into table temp;
+load data inpath '${system:test.tmp.dir}/temp/kv1.txt' overwrite into table target;
+select count(*) from target;
+
+load data local inpath '../../data/files/kv2.txt' into table temp;
+load data inpath '${system:test.tmp.dir}/temp/kv2.txt' overwrite into table target;
+select count(*) from target;
+
+drop table target;
+drop table temp;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q
index cce297cca46dba23b5be7710e12a6c01346ac5aa..55ac1a8185a462f82b9d9e6c846cd12763c3242f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q
@@ -1,9 +1,10 @@
-dfs -mkdir hdfs:///tmp/test/;
+dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name/;
 
-dfs -copyFromLocal ../data/files hdfs:///tmp/test/.;
+dfs -copyFromLocal ../../data/files hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name/.;
 
 CREATE TABLE load_file_with_space_in_the_name(name STRING, age INT);
-LOAD DATA INPATH 'hdfs:///tmp/test/files/person age.txt' INTO TABLE load_file_with_space_in_the_name;
+LOAD DATA INPATH 'hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name/files/person age.txt' INTO TABLE load_file_with_space_in_the_name;
+LOAD DATA INPATH 'hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name/files/person+age.txt' INTO TABLE load_file_with_space_in_the_name;
 
-dfs -rmr hdfs:///tmp/test;
+dfs -rmr hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q
index 40d8210e57d2a9ba26f580136babcffd2955b1f8..fdee45114bb151ea1db4aaf7e8d53bcdec540f4e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q
@@ -1,4 +1,4 @@
 create table hive_test_src ( col1 string ) stored as textfile;
 set hive.security.authorization.enabled=true;
 grant Update on table hive_test_src to user hive_test_user;
-load data local inpath '../data/files/test.dat' overwrite into table hive_test_src ;
+load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_overwrite.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_overwrite.q
index 73853f15a94bba15c74e813383933aab0807505d..080c78496a65307e273c6d2f3787703a3573a688 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_overwrite.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_overwrite.q
@@ -5,11 +5,11 @@ show table extended like load_overwrite;
 select count(*) from load_overwrite;
 
 
-load data local inpath '../data/files/kv1.txt' into table load_overwrite;
+load data local inpath '../../data/files/kv1.txt' into table load_overwrite;
 show table extended like load_overwrite;
 select count(*) from load_overwrite;
 
 
-load data local inpath '../data/files/kv1.txt' overwrite into table load_overwrite;
+load data local inpath '../../data/files/kv1.txt' overwrite into table load_overwrite;
 show table extended like load_overwrite;
 select count(*) from load_overwrite;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q
index ff54324a5a50cfb76187d5a5d612542ca05b7077..cee5873ca5b9f850dbdc3f237e0b055f11ed8134 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q
@@ -1,4 +1,4 @@
 create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile;
 set hive.security.authorization.enabled=true;
 grant Update on table hive_test_src to user hive_test_user;
-load data local inpath '../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part');
+load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part');
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart1.q
index 0813bb23c37468dc98dd251aa7fe286996e287c9..735befef6f9c93c4138368147b3a8c5538bd4b22 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart1.q
@@ -2,7 +2,7 @@
 
 
 create table hive_test_src ( col1 string ) stored as textfile ;
-load data local inpath '../data/files/test.dat' overwrite into table hive_test_src ;
+load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ;
 
 create table hive_test_dst ( col1 string ) partitioned by ( pcol1 string , pcol2 string) stored as sequencefile;
 insert overwrite table hive_test_dst partition ( pcol1='test_part', pCol2='test_Part') select col1 from hive_test_src ;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart2.q
new file mode 100644
index 0000000000000000000000000000000000000000..a252eaa00d77f73a932dc826449217bfea34bbef
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart2.q
@@ -0,0 +1,9 @@
+
+create table hive_test ( col1 string ) partitioned by ( pcol1 string , pcol2 string) stored as textfile;
+load data local inpath '../../data/files/test.dat' overwrite into table hive_test partition (pcol1='part1',pcol2='part1') ;
+load data local inpath '../../data/files/test.dat' overwrite into table hive_test partition (pcol2='part2',pcol1='part2') ;
+select * from hive_test where pcol1='part1' and pcol2='part1';
+select * from hive_test where pcol1='part2' and pcol2='part2';
+
+
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q
index 6e4df215479bb583e1ec36a3a0643e4dcdf5bfcb..cc9c1fec3bc42ddda143e03e074914d0969952d4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q
@@ -1,6 +1,6 @@
 set hive.cli.errors.ignore=true;
 
-ADD FILE ../data/scripts/error_script;
+ADD FILE ../../data/scripts/error_script;
 
 -- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19, 0.20, 0.20S, 0.23)
 -- (this test is flaky so it is currently disabled for all Hadoop versions)
@@ -14,7 +14,7 @@ FROM src;
 DESCRIBE loadpart1;
 SHOW PARTITIONS loadpart1;
 
-LOAD DATA LOCAL INPATH '../data1/files/kv1.txt' INTO TABLE loadpart1 PARTITION(ds='2009-05-05');
+LOAD DATA LOCAL INPATH '../../data1/files/kv1.txt' INTO TABLE loadpart1 PARTITION(ds='2009-05-05');
 SHOW PARTITIONS loadpart1;
 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q
index fd0f7f2b0cdd45616bb7496d302caa4813ba082c..47b05ff4490fb0e4f23c0cf4fc3ba6da28f7ea0f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 CREATE TEMPORARY MACRO SIGMOID (x DOUBLE) 1.0 / (1.0 + EXP(-x));
 SELECT SIGMOID(2) FROM src LIMIT 1;
 EXPLAIN SELECT SIGMOID(2) FROM src LIMIT 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_addjar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_addjar.q
new file mode 100644
index 0000000000000000000000000000000000000000..f56f0746166782ec3b7fa21662fbd7397bfe5ea9
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_addjar.q
@@ -0,0 +1,14 @@
+
+set hive.auto.convert.join=true;
+set hive.auto.convert.join.use.nonstaged=false;
+
+add jar ${system:maven.local.repository}/org/apache/hive/hcatalog/hive-hcatalog-core/${system:hive.version}/hive-hcatalog-core-${system:hive.version}.jar;
+
+CREATE TABLE t1 (a string, b string)
+ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'
+;
+LOAD DATA LOCAL INPATH "../../data/files/sample.json" INTO TABLE t1;
+select * from src join t1 on src.key =t1.a;
+drop table t1;
+set hive.auto.convert.join=false;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_decimal.q
new file mode 100644
index 0000000000000000000000000000000000000000..b65a7be2d25cd67a61c921730e1b4ffb74dcdf0c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_decimal.q
@@ -0,0 +1,35 @@
+set hive.auto.convert.join=true;
+set hive.auto.convert.join.noconditionaltask=true;
+set hive.auto.convert.join.noconditionaltask.size=10000000;
+
+CREATE TABLE over1k(t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp,
+           dec decimal(4,2),
+           bin binary)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
+STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../../data/files/over1k' OVERWRITE INTO TABLE over1k;
+
+CREATE TABLE t1(dec decimal(4,2)) STORED AS ORC;
+INSERT INTO TABLE t1 select dec from over1k;
+CREATE TABLE t2(dec decimal(4,0)) STORED AS ORC;
+INSERT INTO TABLE t2 select dec from over1k;
+
+explain
+select t1.dec, t2.dec from t1 join t2 on (t1.dec=t2.dec);
+
+set hive.mapjoin.optimized.keys=false;
+
+select t1.dec, t2.dec from t1 join t2 on (t1.dec=t2.dec);
+
+set hive.mapjoin.optimized.keys=true;
+
+select t1.dec, t2.dec from t1 join t2 on (t1.dec=t2.dec);
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q
index d6811d493263f661562bf40d6cc6292b33c8c075..a9e1960a5bb05a6a1ee1255e8e65c71c86645f21 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q
@@ -1,4 +1,5 @@
-set hive.exec.post.hooks = org.apache.hadoop.hive.ql.hooks.MapJoinCounterHook ;
+set hive.exec.post.hooks = org.apache.hadoop.hive.ql.hooks.MapJoinCounterHook,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook;
+
 drop table dest1;
 CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q
index 3f87db28ed2e281df5d4baa969f7441f60289954..1eb95f637866987eb0f2eb4a4364f1988672980f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q
@@ -6,6 +6,14 @@ set hive.auto.convert.join.noconditionaltask.size=10000;
 
 explain select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key);
 
-explain select count(*) from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) group by ds;
+explain
+select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) where srcpart.value > 'val_450';
+select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) where srcpart.value > 'val_450';
+
+explain
+select count(*) from srcpart join src on (srcpart.value=src.value) join src src1 on (srcpart.key=src1.key) group by ds;
+select count(*) from srcpart join src on (srcpart.value=src.value) join src src1 on (srcpart.key=src1.key) group by ds;
+
+set hive.mapjoin.lazy.hashtable=false;
 
 select count(*) from srcpart join src src on (srcpart.value=src.value) join src src1 on (srcpart.key=src1.key) group by ds;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_memcheck.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_memcheck.q
new file mode 100644
index 0000000000000000000000000000000000000000..b23361724e669deef2700f83ec827d181237e18c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_memcheck.q
@@ -0,0 +1,16 @@
+
+set hive.auto.convert.join = true;
+
+create table src0 like src;
+insert into table src0 select * from src where src.key < 10;
+
+set hive.mapjoin.check.memory.rows=1;
+
+explain 
+select src1.key as k1, src1.value as v1, src2.key, src2.value
+from src0 src1 inner join src0 src2 on src1.key = src2.key order by k1, v1;
+
+select src1.key as k1, src1.value as v1, src2.key, src2.value
+from src0 src1 inner join src0 src2 on src1.key = src2.key order by k1, v1;
+
+drop table src0;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q
index 9980946057fe10ef9925d4d70a544ff0bca996d2..aed89905238b2c4a23891185b65e7a92e725e645 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q
@@ -11,9 +11,9 @@ ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t';
 CREATE TABLE z (id INT, name STRING)
 ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t';
 
-load data local inpath '../data/files/x.txt' INTO TABLE x;
-load data local inpath '../data/files/y.txt' INTO TABLE y;
-load data local inpath '../data/files/z.txt' INTO TABLE z;
+load data local inpath '../../data/files/x.txt' INTO TABLE x;
+load data local inpath '../../data/files/y.txt' INTO TABLE y;
+load data local inpath '../../data/files/z.txt' INTO TABLE z;
 
 set hive.auto.convert.join=true;
 set hive.auto.convert.join.noconditionaltask=true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q
index aacd0cd68fc5068e9b313c22bfb0869ecae7f803..c5c7ea202fa7631f1d26d9007a439d191dde1a3d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q
@@ -37,7 +37,7 @@ where ds is not null;
 
 show partitions merge_src_part2;
 
-select * from merge_src_part2 where ds is not null ORDER BY key ASC, value ASC;
+select * from merge_src_part2 where ds is not null ORDER BY key ASC, value ASC, ds ASC;
 
 drop table merge_src_part2;
 
@@ -54,4 +54,4 @@ select key, value, ds;
 
 show partitions merge_src_part2;
 
-select * from merge_src_part2 where ds is not null ORDER BY key ASC, value ASC;
+select * from merge_src_part2 where ds is not null ORDER BY key ASC, value ASC, ds ASC;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q
index 744783bd62f8db448a0c3d305d6014e9daf2c224..5a167aa0f08b373802cc9563c5a19e5aec7685ad 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q
@@ -11,14 +11,14 @@ insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select key, v
 
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select key, value, hr from srcpart where ds='2008-04-08';
 
-select * from nzhang_part;
+select * from nzhang_part ORDER BY key, value, ds, hr;
 
 explain
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select key, value from srcpart where ds='2008-04-08';
 
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select key, value from srcpart where ds='2008-04-08';
 
-select * from nzhang_part;
+select * from nzhang_part ORDER BY key, value, ds, hr;
 
 explain
 insert overwrite table nzhang_part partition (ds='2010-08-15', hr) 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q
index 1379426b601f0a446290586a8f35c849183f7997..ae319865f6e7bba2f955b0b97b4cabc7084a12bb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q
@@ -5,15 +5,16 @@ create table srcpart_merge_dp like srcpart;
 
 create table merge_dynamic_part like srcpart;
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
 
 set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; 
 set hive.merge.mapfiles=false;
 set hive.merge.mapredfiles=false;
 set hive.merge.smallfiles.avgsize=1000000000;
+set hive.optimize.sort.dynamic.partition=false;
 explain
 insert overwrite table merge_dynamic_part partition (ds='2008-04-08', hr) select key, value, hr from srcpart_merge_dp where ds='2008-04-08';
 insert overwrite table merge_dynamic_part partition (ds='2008-04-08', hr) select key, value, hr from srcpart_merge_dp where ds='2008-04-08';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q
index b51c70ed03a4619b799cf6f613b79946705a5128..73a71e6265d508a34662b676019ce88bfd5115a4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q
@@ -5,12 +5,12 @@ create table srcpart_merge_dp like srcpart;
 
 create table merge_dynamic_part like srcpart;
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket0.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
-load data local inpath '../data/files/srcbucket1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket0.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
+load data local inpath '../../data/files/srcbucket1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
 
 
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; 
@@ -18,6 +18,7 @@ set hive.merge.mapfiles=true;
 set hive.merge.mapredfiles=true;
 set hive.merge.smallfiles.avgsize=3000;
 set hive.exec.compress.output=false;
+set hive.optimize.sort.dynamic.partition=false;
 
 explain
 insert overwrite table merge_dynamic_part partition (ds='2008-04-08', hr) select key, value, hr from srcpart_merge_dp where ds='2008-04-08';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q
index b3bcf01ea043d43a558f32d7fdeccddc4ed0a68b..43be59e5d5f40a42c45f2791d9ba4711d42df83b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q
@@ -5,20 +5,20 @@ create table srcpart_merge_dp like srcpart;
 
 create table merge_dynamic_part like srcpart;
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
-
-load data local inpath '../data/files/kv1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=11);
-load data local inpath '../data/files/kv2.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=11);
-load data local inpath '../data/files/kv1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=12);
-load data local inpath '../data/files/kv2.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=12);
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
+
+load data local inpath '../../data/files/kv1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=11);
+load data local inpath '../../data/files/kv2.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=11);
+load data local inpath '../../data/files/kv1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=12);
+load data local inpath '../../data/files/kv2.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=12);
 
 show partitions srcpart_merge_dp;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q
index ef769a042d7cef2a2f8f365c56ceb66b13692af9..589717096a9d9f210d36745382cee28df8508e1d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q
@@ -9,15 +9,15 @@ alter table srcpart_merge_dp_rc set fileformat RCFILE;
 create table merge_dynamic_part like srcpart;
 alter table merge_dynamic_part set fileformat RCFILE;
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
 
 insert overwrite table srcpart_merge_dp_rc partition (ds = '2008-04-08', hr) 
 select key, value, hr from srcpart_merge_dp where ds = '2008-04-08';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q
index a196fa05288b77059834b46fc59aef3935ff2208..9f6472456386520fb66983f2650d08ed4cf0d9c7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q
@@ -8,12 +8,12 @@ alter table srcpart_merge_dp_rc set fileformat RCFILE;
 create table merge_dynamic_part like srcpart;
 alter table merge_dynamic_part set fileformat RCFILE;
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11);
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12);
 
 insert overwrite table srcpart_merge_dp_rc partition (ds = '2008-04-08', hr) 
 select key, value, hr from srcpart_merge_dp where ds = '2008-04-08';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q
index 41be152e7871d51658006ccc77424573b2278354..e2da61a78395053be73144cc74ead50fc246b6f1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q
@@ -1,8 +1,8 @@
 create table tmp_meta_export_listener_drop_test (foo string);
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/HIVE-3427;
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/data/exports/HIVE-3427;
 set hive.metastore.pre.event.listeners=org.apache.hadoop.hive.ql.parse.MetaDataExportListener;
-set hive.metadata.export.location=../build/ql/test/data/exports/HIVE-3427;
+set hive.metadata.export.location=${system:test.tmp.dir}/data/exports/HIVE-3427;
 set hive.move.exported.metadata.to.trash=false;
 drop table tmp_meta_export_listener_drop_test;
-dfs -rmr ../build/ql/test/data/exports/HIVE-3427;
+dfs -rmr ${system:test.tmp.dir}/data/exports/HIVE-3427;
 set hive.metastore.pre.event.listeners=;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries.q
new file mode 100644
index 0000000000000000000000000000000000000000..b549a56232108858ce2d6b29a3d5d94237114685
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries.q
@@ -0,0 +1,77 @@
+set hive.stats.dbclass=fs;
+set hive.compute.query.using.stats=true;
+set hive.stats.autogather=true;
+create table over10k(
+           t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp, 
+           dec decimal,  
+           bin binary)
+       row format delimited
+       fields terminated by '|';
+
+load data local inpath '../../data/files/over10k' into table over10k;
+
+create table stats_tbl(
+           t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+	   ts timestamp, 
+           dec decimal,  
+           bin binary);
+
+create table stats_tbl_part(
+           t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+	   ts timestamp, 
+           dec decimal,  
+           bin binary) partitioned by (dt string);
+
+
+insert overwrite table stats_tbl select * from over10k;
+
+insert into table stats_tbl_part partition (dt='2010') select * from over10k where t>0 and t<30;
+insert into table stats_tbl_part partition (dt='2011') select * from over10k where t>30 and t<60;
+insert into table stats_tbl_part partition (dt='2012') select * from over10k where t>60;
+
+explain 
+select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b) from stats_tbl;
+explain
+select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b) from stats_tbl_part;
+
+analyze table stats_tbl compute statistics for columns t,si,i,b,f,d,bo,s,bin;
+analyze table stats_tbl_part partition(dt='2010') compute statistics for columns t,si,i,b,f,d,bo,s,bin;
+analyze table stats_tbl_part partition(dt='2011') compute statistics for columns t,si,i,b,f,d,bo,s,bin;
+analyze table stats_tbl_part partition(dt='2012') compute statistics for columns t,si,i,b,f,d,bo,s,bin;
+
+explain 
+select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl;
+select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl;
+explain 
+select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part;
+select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part;
+
+explain select count(ts) from stats_tbl_part;
+
+drop table stats_tbl;
+drop table stats_tbl_part;
+
+set hive.compute.query.using.stats=false;
+set hive.stats.dbclass=jdbc:derby;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries_with_filters.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries_with_filters.q
new file mode 100644
index 0000000000000000000000000000000000000000..09f4bff616a50f0cf52e108dcb03b459e8bfd1b9
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries_with_filters.q
@@ -0,0 +1,51 @@
+set hive.stats.dbclass=fs;
+set hive.compute.query.using.stats=true;
+create table over10k(
+           t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp, 
+           dec decimal,  
+           bin binary)
+       row format delimited
+       fields terminated by '|';
+
+load data local inpath '../../data/files/over10k' into table over10k;
+
+create table stats_tbl_part(
+           t tinyint,
+           si smallint,
+           i int,
+           b bigint,
+           f float,
+           d double,
+           bo boolean,
+           s string,
+           ts timestamp, 
+           dec decimal,  
+           bin binary) partitioned by (dt int);
+
+
+from over10k 
+insert overwrite table stats_tbl_part partition (dt=2010) select t,si,i,b,f,d,bo,s,ts,dec,bin where t>0 and t<30 
+insert overwrite table stats_tbl_part partition (dt=2014) select t,si,i,b,f,d,bo,s,ts,dec,bin where t > 30 and t<60;
+
+analyze table stats_tbl_part partition(dt) compute statistics;
+analyze table stats_tbl_part partition(dt=2010) compute statistics for columns t,si,i,b,f,d,bo,s,bin;
+analyze table stats_tbl_part partition(dt=2014) compute statistics for columns t,si,i,b,f,d,bo,s,bin;
+
+explain 
+select count(*), count(1), sum(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part where dt = 2010;
+select count(*), count(1), sum(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part where dt = 2010;
+explain 
+select count(*), count(1), sum(1), sum(2), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part where dt > 2010;
+select count(*), count(1), sum(1), sum(2), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part where dt > 2010;
+
+drop table stats_tbl_part;
+set hive.compute.query.using.stats=false;
+set hive.stats.dbclass=jdbc:derby;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q
index 067c143c0e6848bbb20def79d0b47fed978aa862..2a6059b3a37ff8ad9185149482da5c1e822c56e4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q
@@ -15,7 +15,7 @@ GROUP BY key, value, ds, hr;
 show partitions nzhang_t1;
 show partitions nzhang_t2;
 
-select * from nzhang_t1;
-select * from nzhang_t2;
+select * from nzhang_t1 order by key, value;
+select * from nzhang_t2 order by key, value;
 
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mrr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mrr.q
new file mode 100644
index 0000000000000000000000000000000000000000..9f068cc71394ef0d04d6076d3c6c1a7800d5f280
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mrr.q
@@ -0,0 +1,59 @@
+-- simple query with multiple reduce stages
+EXPLAIN SELECT key, count(value) as cnt FROM src GROUP BY key ORDER BY cnt;
+SELECT key, count(value) as cnt FROM src GROUP BY key ORDER BY cnt;
+
+set hive.auto.convert.join=false;
+-- join query with multiple reduce stages;
+EXPLAIN SELECT s2.key, count(distinct s2.value) as cnt FROM src s1 join src s2 on (s1.key = s2.key) GROUP BY s2.key ORDER BY cnt;
+SELECT s2.key, count(distinct s2.value) as cnt FROM src s1 join src s2 on (s1.key = s2.key) GROUP BY s2.key ORDER BY cnt;
+
+set hive.auto.convert.join=true;
+-- same query with broadcast join
+EXPLAIN SELECT s2.key, count(distinct s2.value) as cnt FROM src s1 join src s2 on (s1.key = s2.key) GROUP BY s2.key ORDER BY cnt;
+SELECT s2.key, count(distinct s2.value) as cnt FROM src s1 join src s2 on (s1.key = s2.key) GROUP BY s2.key ORDER BY cnt;
+
+set hive.auto.convert.join=false;
+-- query with multiple branches in the task dag
+EXPLAIN
+SELECT * 
+FROM
+  (SELECT key, count(value) as cnt 
+  FROM src GROUP BY key ORDER BY cnt) s1
+  JOIN
+  (SELECT key, count(value) as cnt 
+  FROM src GROUP BY key ORDER BY cnt) s2
+  JOIN
+  (SELECT key, count(value) as cnt 
+  FROM src GROUP BY key ORDER BY cnt) s3
+  ON (s1.key = s2.key and s1.key = s3.key)
+WHERE
+  s1.cnt > 1
+ORDER BY s1.key;
+
+SELECT * 
+FROM
+  (SELECT key, count(value) as cnt 
+  FROM src GROUP BY key ORDER BY cnt) s1
+  JOIN
+  (SELECT key, count(value) as cnt 
+  FROM src GROUP BY key ORDER BY cnt) s2
+  JOIN
+  (SELECT key, count(value) as cnt 
+  FROM src GROUP BY key ORDER BY cnt) s3
+  ON (s1.key = s2.key and s1.key = s3.key)
+WHERE
+  s1.cnt > 1
+ORDER BY s1.key;
+
+set hive.auto.convert.join=true;
+-- query with broadcast join in the reduce stage
+EXPLAIN
+SELECT *
+FROM
+  (SELECT key, count(value) as cnt FROM src GROUP BY key) s1
+  JOIN src ON (s1.key = src.key);
+
+SELECT *
+FROM
+  (SELECT key, count(value) as cnt FROM src GROUP BY key) s1
+  JOIN src ON (s1.key = src.key);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q
index 9a0a792a91897dcabdfe673ecc63de6970158311..455f550ae3ac5a8f9ea3edced0033646bec8ac3e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q
@@ -1,3 +1,5 @@
+set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook;
+
 create table smallTbl1(key string, value string);
 insert overwrite table smallTbl1 select * from src where key < 10;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q
index ce6cf6d8d6c0d5a29a51cce3e7f1a76f23098dcb..141db4db0a3f306c051ef21bb9ee2ce288006560 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q
@@ -1,3 +1,4 @@
+set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook;
 set hive.auto.convert.join=true;
 set hive.auto.convert.join.noconditionaltask=true;
 set hive.auto.convert.join.noconditionaltask.size=6000;
@@ -187,3 +188,28 @@ FROM part_table x JOIN src1 y ON (x.key = y.key);
 SELECT count(*)
 FROM part_table x JOIN src1 y ON (x.key = y.key);
 
+set hive.auto.convert.join.noconditionaltask.size=10000000;
+set hive.optimize.correlation=false;
+-- HIVE-5891 Alias conflict when merging multiple mapjoin tasks into their common
+-- child mapred task
+EXPLAIN   
+SELECT * FROM (
+  SELECT c.key FROM
+    (SELECT a.key FROM src a JOIN src b ON a.key=b.key GROUP BY a.key) tmp
+    JOIN src c ON tmp.key=c.key
+  UNION ALL
+  SELECT c.key FROM
+    (SELECT a.key FROM src a JOIN src b ON a.key=b.key GROUP BY a.key) tmp
+    JOIN src c ON tmp.key=c.key
+) x;
+
+SELECT * FROM (
+  SELECT c.key FROM
+    (SELECT a.key FROM src a JOIN src b ON a.key=b.key GROUP BY a.key) tmp
+    JOIN src c ON tmp.key=c.key
+  UNION ALL
+  SELECT c.key FROM
+    (SELECT a.key FROM src a JOIN src b ON a.key=b.key GROUP BY a.key) tmp
+    JOIN src c ON tmp.key=c.key
+) x;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q
index b94fbb7b8be4384fba0207b3268ffb99c51f6609..6fd76b859e4e4624e49d6ca02aa5c4c40c1b5d6f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q
@@ -17,6 +17,6 @@ describe nestedcomplex;
 describe extended nestedcomplex;
 
 
-load data local inpath '../data/files/nested_complex.txt' overwrite into table nestedcomplex;
+load data local inpath '../../data/files/nested_complex.txt' overwrite into table nestedcomplex;
 
 select * from nestedcomplex sort by simple_int;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/newline.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/newline.q
index 722ecf6d972cabf4636c280e23d208c924154825..11168fcd3b05c8e0ceaad85e3da086197300dd7f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/newline.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/newline.q
@@ -1,4 +1,4 @@
-add file ../data/scripts/newline.py;
+add file ../../data/scripts/newline.py;
 set hive.transform.escape.input=true;
 
 create table tmp_tmp(key string, value string) stored as rcfile;
@@ -10,10 +10,10 @@ select * from tmp_tmp ORDER BY key ASC, value ASC;
 
 drop table tmp_tmp;
 
-add file ../data/scripts/escapednewline.py;
-add file ../data/scripts/escapedtab.py;
-add file ../data/scripts/doubleescapedtab.py;
-add file ../data/scripts/escapedcarriagereturn.py;
+add file ../../data/scripts/escapednewline.py;
+add file ../../data/scripts/escapedtab.py;
+add file ../../data/scripts/doubleescapedtab.py;
+add file ../../data/scripts/escapedcarriagereturn.py;
 
 create table tmp_tmp(key string, value string) stored as rcfile;
 insert overwrite table tmp_tmp
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonmr_fetch_threshold.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonmr_fetch_threshold.q
new file mode 100644
index 0000000000000000000000000000000000000000..e6343e2f535c02047f7e0f11175c69631902a24a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonmr_fetch_threshold.q
@@ -0,0 +1,9 @@
+set hive.fetch.task.conversion=more;
+
+explain select * from srcpart where ds='2008-04-08' AND hr='11' limit 10;
+explain select cast(key as int) * 10, upper(value) from src limit 10;
+
+set hive.fetch.task.conversion.threshold=100;
+
+explain select * from srcpart where ds='2008-04-08' AND hr='11' limit 10;
+explain select cast(key as int) * 10, upper(value) from src limit 10;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q
index 9cb89da373d2923e61dc311393a25aba1d029d66..e33b4bfcab7baa18854e47795f96c7485cc280ea 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q
@@ -1,12 +1,12 @@
 CREATE TABLE table(string string) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/docurl.txt' INTO TABLE table;
+LOAD DATA LOCAL INPATH '../../data/files/docurl.txt' INTO TABLE table;
 
 SELECT table, count(1)
 FROM
 (
   FROM table
   SELECT TRANSFORM (table.string)
-  USING 'java -cp ../build/ql/test/classes org.apache.hadoop.hive.scripts.extracturl' AS (table, count)
+  USING 'java -cp ../util/target/classes/ org.apache.hadoop.hive.scripts.extracturl' AS (table, count)
 ) subq
 GROUP BY table;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias3.q
new file mode 100644
index 0000000000000000000000000000000000000000..aa79674409d2d37a59691c29d0ccd92f6c097ec0
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias3.q
@@ -0,0 +1,4 @@
+CREATE TABLE dest1(c string, key INT, value DOUBLE) STORED AS TEXTFILE;
+
+FROM src
+INSERT OVERWRITE TABLE dest1 SELECT '1234', src.key, sum(src.value) WHERE src.key < 100 group by key;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q
index 48c39b81fdd8f409e254ecc2261a025b324a8ff4..bd0cb8d12d88fa6b416bd317dfecd06b30298b4d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q
@@ -2,10 +2,10 @@ EXPLAIN SELECT ARRAY(NULL, 0),
                ARRAY(NULL, ARRAY()),
                ARRAY(NULL, MAP()),
                ARRAY(NULL, STRUCT(0))
-        FROM src LIMIT 1;
+        FROM src tablesample (1 rows);
 
 SELECT ARRAY(NULL, 0), 
        ARRAY(NULL, ARRAY()),
        ARRAY(NULL, MAP()),
        ARRAY(NULL, STRUCT(0))
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q
index fa4a8639446e59317687f1f5b33409fb2be19d78..4b43d608e86df2b9880295af6edf334f884911a3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q
@@ -3,7 +3,7 @@
 
 
 create table temp_null(a int) stored as textfile;
-load data local inpath '../data/files/test.dat' overwrite into table temp_null;
+load data local inpath '../../data/files/test.dat' overwrite into table temp_null;
 
 select null, null from temp_null;
 
@@ -15,8 +15,8 @@ create table tt_b(a int, b string) row format serde "org.apache.hadoop.hive.serd
 insert overwrite table tt_b select null, null from temp_null;
 select * from tt_b;
 
-insert overwrite directory "../build/ql/test/data/warehouse/null_columns.out" select null, null from temp_null;
-dfs -cat ../build/ql/test/data/warehouse/null_columns.out/*;
+insert overwrite directory "target/warehouse/null_columns.out" select null, null from temp_null;
+dfs -cat ${system:test.warehouse.dir}/null_columns.out/*;
 
 
 create table temp_null2 (key string, value string) partitioned by (ds string);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformat.q
new file mode 100644
index 0000000000000000000000000000000000000000..c9a7dab5eb9e77a5e6a11326b033b8090dbbc21d
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformat.q
@@ -0,0 +1,24 @@
+-- base table with null data
+DROP TABLE IF EXISTS base_tab;
+CREATE TABLE base_tab(a STRING, b STRING, c STRING, d STRING) STORED AS TEXTFILE;
+LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE base_tab;
+DESCRIBE EXTENDED base_tab;
+
+-- table with non-default null format
+DROP TABLE IF EXISTS null_tab1;
+EXPLAIN CREATE TABLE null_tab1(a STRING, b STRING) ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull';
+CREATE TABLE null_tab1(a STRING, b STRING) ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull';
+DESCRIBE EXTENDED null_tab1;
+SHOW CREATE TABLE null_tab1;
+
+-- load null data from another table and verify that the null is stored in the expected format
+INSERT OVERWRITE TABLE null_tab1 SELECT a,b FROM base_tab;
+dfs -cat ${system:test.warehouse.dir}/null_tab1/*;
+SELECT * FROM null_tab1;
+-- alter the null format and verify that the old null format is no longer in effect
+ALTER TABLE null_tab1 SET SERDEPROPERTIES ( 'serialization.null.format'='foo');
+SELECT * FROM null_tab1;
+
+
+DROP TABLE null_tab1;
+DROP TABLE base_tab;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatCTAS.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatCTAS.q
new file mode 100644
index 0000000000000000000000000000000000000000..d077981d02ad093daedcb3ac2b92b3aecfd39d3c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatCTAS.q
@@ -0,0 +1,24 @@
+-- base table with null data
+DROP TABLE IF EXISTS base_tab;
+CREATE TABLE base_tab(a STRING, b STRING, c STRING, d STRING) STORED AS TEXTFILE;
+LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE base_tab;
+DESCRIBE EXTENDED base_tab;
+
+-- table with non-default null format
+DROP TABLE IF EXISTS null_tab3;
+EXPLAIN CREATE TABLE null_tab3 ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull'
+   AS SELECT a, b FROM base_tab;
+CREATE TABLE null_tab3 ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull'
+   AS SELECT a, b FROM base_tab;
+DESCRIBE EXTENDED null_tab3;
+SHOW CREATE TABLE null_tab3;
+
+dfs -cat ${system:test.warehouse.dir}/null_tab3/*;
+SELECT * FROM null_tab3;
+-- alter the null format and verify that the old null format is no longer in effect
+ALTER TABLE null_tab3 SET SERDEPROPERTIES ( 'serialization.null.format'='foo');
+SELECT * FROM null_tab3;
+
+
+DROP TABLE null_tab3;
+DROP TABLE base_tab;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatdir.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatdir.q
new file mode 100644
index 0000000000000000000000000000000000000000..d29863839f7285f1c39b50857b16a7594382e723
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatdir.q
@@ -0,0 +1,21 @@
+-- base table with null data
+DROP TABLE IF EXISTS base_tab;
+CREATE TABLE base_tab(a STRING, b STRING, c STRING, d STRING) STORED AS TEXTFILE;
+LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE base_tab;
+DESCRIBE EXTENDED base_tab;
+
+dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/hive_test/nullformat/tmp;
+dfs -rmr ${system:test.tmp.dir}/hive_test/nullformat/*;
+INSERT OVERWRITE LOCAL DIRECTORY '${system:test.tmp.dir}/hive_test/nullformat'
+   ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull' SELECT a,b FROM base_tab;
+dfs -cat ${system:test.tmp.dir}/hive_test/nullformat/000000_0;
+
+-- load the exported data back into a table with same null format and verify null values
+DROP TABLE IF EXISTS null_tab2;
+CREATE TABLE null_tab2(a STRING, b STRING) ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull';
+LOAD DATA LOCAL INPATH '${system:test.tmp.dir}/hive_test/nullformat/000000_0' INTO TABLE null_tab2;
+SELECT * FROM null_tab2;
+
+
+dfs -rmr ${system:test.tmp.dir}/hive_test/nullformat;
+DROP TABLE base_tab;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q
index a5bc9ff1f36ffa3819087732c311d93fa7bbfc41..19e5b10786fc01f204bca36e13e11bc063d22335 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q
@@ -1,28 +1,28 @@
 CREATE TABLE tstparttbl(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-09');
-LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-08');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-08');
 explain
 select count(1) from tstparttbl;
 select count(1) from tstparttbl;
 
 CREATE TABLE tstparttbl2(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-09');
-LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-08');
+LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-08');
 explain
 select count(1) from tstparttbl2;
 select count(1) from tstparttbl2;
 DROP TABLE tstparttbl;
 CREATE TABLE tstparttbl(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-09');
-LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-08');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-08');
 explain
 select count(1) from tstparttbl;
 select count(1) from tstparttbl;
 
 DROP TABLE tstparttbl2;
 CREATE TABLE tstparttbl2(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-09');
-LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-08');
+LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-08');
 explain
 select count(1) from tstparttbl2;
 select count(1) from tstparttbl2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q
index 12773b6159a5dbb33a12df987105baea6d02f12c..b4b68fb8b693aab5510809de674381101d78fd54 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q
@@ -1,10 +1,10 @@
 
 CREATE TABLE tstparttbl(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2009-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2009-04-09');
 
 
 CREATE TABLE tstparttbl2(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE tstparttbl2 PARTITION (ds='2009-04-09');
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE tstparttbl2 PARTITION (ds='2009-04-09');
 
 explain
 select u.* from
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q
index 95c9e1df376792203dd1d82062ed8e209c239a9b..11f4a7a78f813128921a403fb4e573cd6d6559ff 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q
@@ -1,7 +1,7 @@
 
 CREATE TABLE nullscript(KEY STRING, VALUE STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE nullscript;
-LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE nullscript;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE nullscript;
+LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE nullscript;
 explain
 select transform(key) using 'cat' as key1 from nullscript;
 select transform(key) using 'cat' as key1 from nullscript;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q
index 7f858d3e6f136d86e1c975dfa8406d8d6a33a523..d51c2107e1586eb006d21f34adb1d58baf6ac460 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 EXPLAIN SELECT null + 7, 1.0 - null, null + null,
                CAST(21 AS BIGINT) % CAST(5 AS TINYINT),
                CAST(21 AS BIGINT) % CAST(21 AS BIGINT),
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q
index b685ae6c4e53f5bfcd8b55481caadfc69a6d65f5..ec9e8076fd5336ba4b3ff5eb2c2eff990238c7c2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q
@@ -1,3 +1,4 @@
+set hive.fetch.task.conversion=more;
 
 select 1.0 < 2.0 from src limit 1;
 select 2.0 < 2.0 from src limit 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optrstat_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optrstat_groupby.q
deleted file mode 100644
index 5993041405ed856cea716173660552c50ec7a64e..0000000000000000000000000000000000000000
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optrstat_groupby.q
+++ /dev/null
@@ -1,6 +0,0 @@
-SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.OptrStatGroupByHook;
-SET hive.exec.mode.local.auto=false;
-SET hive.task.progress=true;
--- This test executes the OptrStatGroupBy hook which prints the optr level
--- stats of GROUPBY optr present is the plan of below query
-SELECT count(1) FROM src;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_analyze.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_analyze.q
new file mode 100644
index 0000000000000000000000000000000000000000..915f4f0d71e63ced9029ca8ee27d5b8ae4c48872
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_analyze.q
@@ -0,0 +1,179 @@
+CREATE TABLE orc_create_people_staging (
+  id int,
+  first_name string,
+  last_name string,
+  address string,
+  salary decimal,
+  start_date timestamp,
+  state string);
+
+LOAD DATA LOCAL INPATH '../../data/files/orc_create_people.txt' OVERWRITE INTO TABLE orc_create_people_staging;
+
+set hive.exec.dynamic.partition.mode=nonstrict;
+
+set hive.stats.autogather=false;
+-- non-partitioned table
+-- partial scan gather
+CREATE TABLE orc_create_people (
+  id int,
+  first_name string,
+  last_name string,
+  address string,
+  salary decimal,
+  start_date timestamp,
+  state string)
+STORED AS orc;
+
+INSERT OVERWRITE TABLE orc_create_people SELECT * FROM orc_create_people_staging ORDER BY id;
+
+set hive.stats.autogather = true;
+analyze table orc_create_people compute statistics partialscan;
+
+desc formatted orc_create_people;
+
+drop table orc_create_people;
+
+-- auto stats gather
+CREATE TABLE orc_create_people (
+  id int,
+  first_name string,
+  last_name string,
+  address string,
+  salary decimal,
+  start_date timestamp,
+  state string)
+STORED AS orc;
+
+INSERT OVERWRITE TABLE orc_create_people SELECT * FROM orc_create_people_staging ORDER BY id;
+
+desc formatted orc_create_people;
+
+drop table orc_create_people;
+
+set hive.stats.autogather=false;
+-- partitioned table
+-- partial scan gather
+CREATE TABLE orc_create_people (
+  id int,
+  first_name string,
+  last_name string,
+  address string,
+  salary decimal,
+  start_date timestamp)
+PARTITIONED BY (state string)
+STORED AS orc;
+
+INSERT OVERWRITE TABLE orc_create_people PARTITION (state)
+  SELECT * FROM orc_create_people_staging ORDER BY id;
+
+set hive.stats.autogather = true;
+analyze table orc_create_people partition(state) compute statistics partialscan;
+
+desc formatted orc_create_people partition(state="Ca");
+desc formatted orc_create_people partition(state="Or");
+
+drop table orc_create_people;
+
+-- auto stats gather
+CREATE TABLE orc_create_people (
+  id int,
+  first_name string,
+  last_name string,
+  address string,
+  salary decimal,
+  start_date timestamp)
+PARTITIONED BY (state string)
+STORED AS orc;
+
+INSERT OVERWRITE TABLE orc_create_people PARTITION (state)
+  SELECT * FROM orc_create_people_staging ORDER BY id;
+
+desc formatted orc_create_people partition(state="Ca");
+desc formatted orc_create_people partition(state="Or");
+
+drop table orc_create_people;
+
+set hive.stats.autogather=false;
+-- partitioned and bucketed table
+-- partial scan gather
+CREATE TABLE orc_create_people (
+  id int,
+  first_name string,
+  last_name string,
+  address string,
+  salary decimal,
+  start_date timestamp)
+PARTITIONED BY (state string)
+clustered by (first_name)
+sorted by (last_name)
+into 4 buckets
+STORED AS orc;
+
+INSERT OVERWRITE TABLE orc_create_people PARTITION (state)
+  SELECT * FROM orc_create_people_staging ORDER BY id;
+
+set hive.stats.autogather = true;
+analyze table orc_create_people partition(state) compute statistics partialscan;
+
+desc formatted orc_create_people partition(state="Ca");
+desc formatted orc_create_people partition(state="Or");
+
+drop table orc_create_people;
+
+-- auto stats gather
+CREATE TABLE orc_create_people (
+  id int,
+  first_name string,
+  last_name string,
+  address string,
+  salary decimal,
+  start_date timestamp)
+PARTITIONED BY (state string)
+clustered by (first_name)
+sorted by (last_name)
+into 4 buckets
+STORED AS orc;
+
+INSERT OVERWRITE TABLE orc_create_people PARTITION (state)
+  SELECT * FROM orc_create_people_staging ORDER BY id;
+
+desc formatted orc_create_people partition(state="Ca");
+desc formatted orc_create_people partition(state="Or");
+
+drop table orc_create_people;
+
+set hive.stats.autogather=false;
+-- create table with partitions containing text and ORC files.
+-- ORC files implements StatsProvidingRecordReader but text files does not.
+-- So the partition containing text file should not have statistics.
+CREATE TABLE orc_create_people (
+  id int,
+  first_name string,
+  last_name string,
+  address string,
+  salary decimal,
+  start_date timestamp)
+PARTITIONED BY (state string)
+STORED AS orc;
+
+INSERT OVERWRITE TABLE orc_create_people PARTITION (state)
+  SELECT * FROM orc_create_people_staging ORDER BY id;
+
+-- set the table to text format
+ALTER TABLE orc_create_people SET SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe';
+ALTER TABLE orc_create_people SET FILEFORMAT TEXTFILE;
+
+-- load the text data into a new partition
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE orc_create_people PARTITION(state="OH");
+
+-- set the table back to orc
+ALTER TABLE orc_create_people SET SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde';
+ALTER TABLE orc_create_people SET FILEFORMAT ORC;
+
+set hive.stats.autogather = true;
+analyze table orc_create_people partition(state) compute statistics noscan;
+
+desc formatted orc_create_people partition(state="Ca");
+desc formatted orc_create_people partition(state="OH");
+
+drop table orc_create_people;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q
index 6aca5486445c809c54857929f5a17cb291f74093..a82c1a55d5f6c61f382441d823ac5ec073831b18 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q
@@ -40,7 +40,7 @@ set hive.default.fileformat=orc;
 CREATE TABLE orc_create (key INT, value STRING)
    PARTITIONED BY (ds string);
 
-set hive.default.fileformat=text;
+set hive.default.fileformat=TextFile;
 
 DESCRIBE FORMATTED orc_create;
 
@@ -53,7 +53,7 @@ CREATE TABLE orc_create_complex (
 
 DESCRIBE FORMATTED orc_create_complex;
 
-LOAD DATA LOCAL INPATH '../data/files/orc_create.txt' OVERWRITE INTO TABLE orc_create_staging;
+LOAD DATA LOCAL INPATH '../../data/files/orc_create.txt' OVERWRITE INTO TABLE orc_create_staging;
 
 SELECT * from orc_create_staging;
 
@@ -70,16 +70,20 @@ CREATE TABLE orc_create_people_staging (
   first_name string,
   last_name string,
   address string,
+  salary decimal,
+  start_date timestamp,
   state string);
 
-LOAD DATA LOCAL INPATH '../data/files/orc_create_people.txt'
+LOAD DATA LOCAL INPATH '../../data/files/orc_create_people.txt'
   OVERWRITE INTO TABLE orc_create_people_staging;
 
 CREATE TABLE orc_create_people (
   id int,
   first_name string,
   last_name string,
-  address string)
+  address string,
+  salary decimal,
+  start_date timestamp)
 PARTITIONED BY (state string)
 STORED AS orc;
 
@@ -92,9 +96,26 @@ SET hive.optimize.index.filter=true;
 -- test predicate push down with partition pruning
 SELECT COUNT(*) FROM orc_create_people where id < 10 and state = 'Ca';
 
+-- test predicate push down
+SELECT COUNT(*) FROM orc_create_people where id = 50;
+SELECT COUNT(*) FROM orc_create_people where id between 10 and 20;
+SELECT COUNT(*) FROM orc_create_people where id > 10 and id < 100;
+SELECT COUNT(*) FROM orc_create_people where (id + 1) = 20;
+SELECT COUNT(*) FROM orc_create_people where (id + 10) < 200;
+SELECT COUNT(*) FROM orc_create_people where id < 30  or first_name = "Rafael";
+SELECT COUNT(*) FROM orc_create_people 
+   where length(substr(first_name, 1, 2)) <= 2 and last_name like '%';
+SELECT COUNT(*) FROM orc_create_people where salary = 200.00;
+SELECT COUNT(*) FROM orc_create_people WHERE start_date IS NULL;
+SELECT COUNT(*) FROM orc_create_people WHERE YEAR(start_date) = 2014;
+
+-- test predicate push down with partition pruning
+SELECT COUNT(*) FROM orc_create_people where salary = 200.00 and state = 'Ca';
+
 -- test predicate push down with no column projection
 SELECT id, first_name, last_name, address
-  FROM orc_create_people WHERE id > 90;
+  FROM orc_create_people WHERE id > 90
+  ORDER BY id, first_name, last_name;
 
 DROP TABLE orc_create;
 DROP TABLE orc_create_complex;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q
index f916012b8365e1f77bb9d3bb2b1101904712d663..a0eaab75fac3eaf41bec3679c1a53a6f97b3cd2b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q
@@ -19,7 +19,7 @@ SELECT * FROM test_orc;
 ALTER TABLE test_orc SET SERDEPROPERTIES ('orc.stripe.size' = '1');
 
 CREATE TABLE src_thousand(key STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1kv2.cogroup.txt' 
+LOAD DATA LOCAL INPATH '../../data/files/kv1kv2.cogroup.txt' 
      INTO TABLE src_thousand;
 
 set hive.exec.orc.dictionary.key.size.threshold=0.5;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q
index cbfd7b359f8782960c99e36e494340016dfb8ccb..0c8861e41c4bb836b67f35106c7e793fed62b4e2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q
@@ -10,7 +10,7 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
 -- to another partition
 -- This can produce unexpected results with CombineHiveInputFormat
 
-INSERT OVERWRITE TABLE test_orc PARTITION (part = '1') SELECT key FROM src LIMIT 5;
+INSERT OVERWRITE TABLE test_orc PARTITION (part = '1') SELECT key FROM src tablesample (5 rows);
 
 ALTER TABLE test_orc ADD COLUMNS (cnt INT);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols2.q
new file mode 100644
index 0000000000000000000000000000000000000000..f7e80a75b3ba670097c398588c4eb2be7e47c182
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols2.q
@@ -0,0 +1,11 @@
+-- Create a table with one column, write to it, then add an additional column
+-- This can break reads
+
+CREATE TABLE test_orc (key STRING)
+STORED AS ORC;
+
+INSERT OVERWRITE TABLE test_orc SELECT key FROM src LIMIT 5;
+
+ALTER TABLE test_orc ADD COLUMNS (value STRING);
+
+SELECT * FROM test_orc order by key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q
index 0ef57d18ccaf7ab43823c568ab2893b65380285e..34cd6d47dfcd9322b527e94565acf5184c7fd218 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q
@@ -3,13 +3,13 @@ ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde'
 STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' 
 OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat';
 
-INSERT OVERWRITE TABLE test_orc SELECT '' FROM src limit 10;
+INSERT OVERWRITE TABLE test_orc SELECT '' FROM src tablesample (10 rows);
 
 -- Test reading a column which is just empty strings
 
 SELECT * FROM test_orc; 
 
-INSERT OVERWRITE TABLE test_orc SELECT IF (key % 3 = 0, key, '') FROM src limit 10;
+INSERT OVERWRITE TABLE test_orc SELECT IF (key % 3 = 0, key, '') FROM src tablesample (10 rows);
 
 -- Test reading a column which has some empty strings
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q
index 6685da7a82245e5c6889e6b0f3419181217d05ee..83c5a0505e57ed984176559bb933991e87962622 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q
@@ -10,7 +10,7 @@ ALTER TABLE test_orc SET SERDEPROPERTIES ('orc.row.index.stride' = '1000');
 -- to last index stride are the same (there's only two index strides)
 
 CREATE TABLE src_null(a STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/nulls.txt' INTO TABLE src_null;
+LOAD DATA LOCAL INPATH '../../data/files/nulls.txt' INTO TABLE src_null;
 
 INSERT OVERWRITE TABLE test_orc SELECT a FROM src_null;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_min_max.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_min_max.q
new file mode 100644
index 0000000000000000000000000000000000000000..b81adf2af3e535aa6c17710392fe9ba5946899d6
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_min_max.q
@@ -0,0 +1,32 @@
+create table if not exists alltypes (
+ bo boolean,
+ ti tinyint,
+ si smallint,
+ i int,
+ bi bigint,
+ f float,
+ d double,
+ de decimal(10,3),
+ ts timestamp,
+ da date,
+ s string,
+ c char(5),
+ vc varchar(5),
+ m map<string, string>,
+ l array<int>,
+ st struct<c1:int, c2:string>
+) row format delimited fields terminated by '|'
+collection items terminated by ','
+map keys terminated by ':' stored as textfile;
+
+create table alltypes_orc like alltypes;
+alter table alltypes_orc set fileformat orc;
+
+load data local inpath '../../data/files/alltypes2.txt' overwrite into table alltypes;
+
+insert overwrite table alltypes_orc select * from alltypes;
+
+select min(bo), max(bo), min(ti), max(ti), min(si), max(si), min(i), max(i), min(bi), max(bi), min(f), max(f), min(d), max(d), min(de), max(de), min(ts), max(ts), min(da), max(da), min(s), max(s), min(c), max(c), min(vc), max(vc) from alltypes;
+
+select min(bo), max(bo), min(ti), max(ti), min(si), max(si), min(i), max(i), min(bi), max(bi), min(f), max(f), min(d), max(d), min(de), max(de), min(ts), max(ts), min(da), max(da), min(s), max(s), min(c), max(c), min(vc), max(vc) from alltypes_orc;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q
new file mode 100644
index 0000000000000000000000000000000000000000..1f5f54ae19ee8035505d5aaf264fded6c82b7514
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q
@@ -0,0 +1,76 @@
+SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+SET mapred.min.split.size=1000;
+SET mapred.max.split.size=5000;
+
+create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); 
+
+insert overwrite table newtypesorc select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl;
+
+set hive.optimize.index.filter=false;
+
+-- char data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
+select sum(hash(*)) from newtypesorc where c="apple";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where c="apple";
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where c!="apple";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where c!="apple";
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where c<"hello";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where c<"hello";
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where c<="hello";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where c<="hello";
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where c="apple ";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where c="apple ";
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where c in ("apple", "carrot");
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where c in ("apple", "carrot");
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where c in ("apple", "hello");
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where c in ("apple", "hello");
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where c in ("carrot");
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where c in ("carrot");
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where c between "apple" and "carrot";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where c between "apple" and "carrot";
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where c between "apple" and "zombie";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where c between "apple" and "zombie";
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where c between "carrot" and "carrot1";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where c between "carrot" and "carrot1";
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q
new file mode 100644
index 0000000000000000000000000000000000000000..c34be867e484f40dc074fd0579fa96bec40f7027
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q
@@ -0,0 +1,97 @@
+SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+SET mapred.min.split.size=1000;
+SET mapred.max.split.size=5000;
+
+create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); 
+
+insert overwrite table newtypesorc select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl;
+
+-- date data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
+select sum(hash(*)) from newtypesorc where da='1970-02-20';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da='1970-02-20';
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where da=cast('1970-02-20' as date);
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da=cast('1970-02-20' as date);
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where da=cast('1970-02-20' as varchar(20));
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da=cast('1970-02-20' as varchar(20));
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where da!='1970-02-20';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da!='1970-02-20';
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where da<'1970-02-27';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da<'1970-02-27';
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where da<'1970-02-29';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da<'1970-02-29';
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where da<'1970-02-15';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da<'1970-02-15';
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where da<='1970-02-20';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da<='1970-02-20';
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where da<='1970-02-27';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da<='1970-02-27';
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where da in (cast('1970-02-21' as date), cast('1970-02-27' as date));
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da in (cast('1970-02-21' as date), cast('1970-02-27' as date));
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where da in (cast('1970-02-20' as date), cast('1970-02-27' as date));
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da in (cast('1970-02-20' as date), cast('1970-02-27' as date));
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where da in (cast('1970-02-21' as date), cast('1970-02-22' as date));
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da in (cast('1970-02-21' as date), cast('1970-02-22' as date));
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where da between '1970-02-19' and '1970-02-22';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da between '1970-02-19' and '1970-02-22';
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where da between '1970-02-19' and '1970-02-28';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da between '1970-02-19' and '1970-02-28';
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where da between '1970-02-18' and '1970-02-19';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where da between '1970-02-18' and '1970-02-19';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q
new file mode 100644
index 0000000000000000000000000000000000000000..a93590eacca01b2d752303eda2a1a8df6caacb47
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q
@@ -0,0 +1,151 @@
+SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+SET mapred.min.split.size=1000;
+SET mapred.max.split.size=5000;
+
+create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); 
+
+insert overwrite table newtypesorc select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl;
+
+-- decimal data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
+select sum(hash(*)) from newtypesorc where d=0.22;
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d=0.22;
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d='0.22';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d='0.22';
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d=cast('0.22' as float);
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d=cast('0.22' as float);
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d!=0.22;
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d!=0.22;
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d!='0.22';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d!='0.22';
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d!=cast('0.22' as float);
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d!=cast('0.22' as float);
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d<11.22;
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d<11.22;
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d<'11.22';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d<'11.22';
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d<cast('11.22' as float);
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d<cast('11.22' as float);
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d<1;
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d<1;
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d<=11.22;
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d<=11.22;
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d<='11.22';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d<='11.22';
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d<=cast('11.22' as float);
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d<=cast('11.22' as float);
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d<=12;
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d<=12;
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d in ('0.22', '1.0');
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d in ('0.22', '1.0');
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d in ('0.22', '11.22');
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d in ('0.22', '11.22');
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d in ('0.9', '1.0');
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d in ('0.9', '1.0');
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d in ('0.9', 0.22);
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d in ('0.9', 0.22);
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d in ('0.9', 0.22, cast('11.22' as float));
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d in ('0.9', 0.22, cast('11.22' as float));
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d between 0 and 1;
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d between 0 and 1;
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d between 0 and 1000;
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d between 0 and 1000;
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d between 0 and '2.0';
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d between 0 and '2.0';
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d between 0 and cast(3 as float);
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d between 0 and cast(3 as float);
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where d between 1 and cast(30 as char(10));
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where d between 1 and cast(30 as char(10));
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_varchar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_varchar.q
new file mode 100644
index 0000000000000000000000000000000000000000..0fecc664e46db6bfdb8e1b270e3b016da68877ef
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_varchar.q
@@ -0,0 +1,76 @@
+SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+SET mapred.min.split.size=1000;
+SET mapred.max.split.size=5000;
+
+create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); 
+
+insert overwrite table newtypesorc select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl;
+
+set hive.optimize.index.filter=false;
+
+-- varchar data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests)
+select sum(hash(*)) from newtypesorc where v="bee";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where v="bee";
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where v!="bee";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where v!="bee";
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where v<"world";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where v<"world";
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where v<="world";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where v<="world";
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where v="bee   ";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where v="bee   ";
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where v in ("bee", "orange");
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where v in ("bee", "orange");
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where v in ("bee", "world");
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where v in ("bee", "world");
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where v in ("orange");
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where v in ("orange");
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where v between "bee" and "orange";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where v between "bee" and "orange";
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where v between "bee" and "zombie";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where v between "bee" and "zombie";
+
+set hive.optimize.index.filter=false;
+select sum(hash(*)) from newtypesorc where v between "orange" and "pine";
+
+set hive.optimize.index.filter=true;
+select sum(hash(*)) from newtypesorc where v between "orange" and "pine";
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q
index f5f25f00c951ab6bca708cd5171e9acc79f8ebae..a267bfe8e13b91b9383abcaf6353967806a1c391 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q
@@ -7,7 +7,7 @@ CREATE TABLE orc_pred(t tinyint,
            bo boolean,
            s string,
            ts timestamp,
-           dec decimal,
+           dec decimal(4,2),
            bin binary)
 STORED AS ORC;
 
@@ -22,12 +22,12 @@ CREATE TABLE staging(t tinyint,
            bo boolean,
            s string,
            ts timestamp,
-           dec decimal,
+           dec decimal(4,2),
            bin binary)
 ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/over1k' OVERWRITE INTO TABLE staging;
+LOAD DATA LOCAL INPATH '../../data/files/over1k' OVERWRITE INTO TABLE staging;
 
 INSERT INTO TABLE orc_pred select * from staging;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_split_elimination.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_split_elimination.q
new file mode 100644
index 0000000000000000000000000000000000000000..54eb23e776b88e1921d4b14ed316cdd3bc7addfa
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_split_elimination.q
@@ -0,0 +1,168 @@
+create table orc_split_elim (userid bigint, string1 string, subtype double, decimal1 decimal, ts timestamp) stored as orc;
+
+load data local inpath '../../data/files/orc_split_elim.orc' into table orc_split_elim;
+
+SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+SET mapred.min.split.size=1000;
+SET mapred.max.split.size=5000;
+SET hive.optimize.index.filter=false;
+
+-- The above table will have 5 splits with the followings stats
+--  Stripe 1:
+--    Column 0: count: 5000
+--    Column 1: count: 5000 min: 2 max: 100 sum: 499902
+--    Column 2: count: 5000 min: foo max: zebra sum: 24998
+--    Column 3: count: 5000 min: 0.8 max: 8.0 sum: 39992.8
+--    Column 4: count: 5000 min: 0 max: 1.2 sum: 1.2
+--    Column 5: count: 5000
+--  Stripe 2:
+--    Column 0: count: 5000
+--    Column 1: count: 5000 min: 13 max: 100 sum: 499913
+--    Column 2: count: 5000 min: bar max: zebra sum: 24998
+--    Column 3: count: 5000 min: 8.0 max: 80.0 sum: 40072.0
+--    Column 4: count: 5000 min: 0 max: 2.2 sum: 2.2
+--    Column 5: count: 5000
+--  Stripe 3:
+--    Column 0: count: 5000
+--    Column 1: count: 5000 min: 29 max: 100 sum: 499929
+--    Column 2: count: 5000 min: cat max: zebra sum: 24998
+--    Column 3: count: 5000 min: 8.0 max: 8.0 sum: 40000.0
+--    Column 4: count: 5000 min: 0 max: 3.3 sum: 3.3
+--    Column 5: count: 5000
+--  Stripe 4:
+--    Column 0: count: 5000
+--    Column 1: count: 5000 min: 70 max: 100 sum: 499970
+--    Column 2: count: 5000 min: dog max: zebra sum: 24998
+--    Column 3: count: 5000 min: 1.8 max: 8.0 sum: 39993.8
+--    Column 4: count: 5000 min: 0 max: 4.4 sum: 4.4
+--    Column 5: count: 5000
+--  Stripe 5:
+--    Column 0: count: 5000
+--    Column 1: count: 5000 min: 5 max: 100 sum: 499905
+--    Column 2: count: 5000 min: eat max: zebra sum: 24998
+--    Column 3: count: 5000 min: 0.8 max: 8.0 sum: 39992.8
+--    Column 4: count: 5000 min: 0 max: 5.5 sum: 5.5
+--    Column 5: count: 5000
+
+-- 5 mappers
+select userid,string1,subtype,decimal1,ts from orc_split_elim where userid<=0;
+
+SET hive.optimize.index.filter=true;
+-- 0 mapper
+select userid,string1,subtype,decimal1,ts from orc_split_elim where userid<=0;
+SET hive.optimize.index.filter=false;
+
+-- 5 mappers. count should be 0
+select count(*) from orc_split_elim where userid<=0;
+
+SET hive.optimize.index.filter=true;
+-- 0 mapper
+select count(*) from orc_split_elim where userid<=0;
+SET hive.optimize.index.filter=false;
+
+-- 5 mappers
+select userid,string1,subtype,decimal1,ts from orc_split_elim where userid<=2 order by userid;
+
+SET hive.optimize.index.filter=true;
+-- 1 mapper
+select userid,string1,subtype,decimal1,ts from orc_split_elim where userid<=2 order by userid;
+SET hive.optimize.index.filter=false;
+
+-- 5 mappers
+select userid,string1,subtype,decimal1,ts from orc_split_elim where userid<=5 order by userid;
+
+SET hive.optimize.index.filter=true;
+-- 2 mappers
+select userid,string1,subtype,decimal1,ts from orc_split_elim where userid<=5 order by userid;
+SET hive.optimize.index.filter=false;
+
+-- 5 mappers
+select userid,string1,subtype,decimal1,ts from orc_split_elim where userid<=13 order by userid;
+
+SET hive.optimize.index.filter=true;
+-- 3 mappers
+select userid,string1,subtype,decimal1,ts from orc_split_elim where userid<=13 order by userid;
+SET hive.optimize.index.filter=false;
+
+-- 5 mappers
+select userid,string1,subtype,decimal1,ts from orc_split_elim where userid<=29 order by userid;
+
+SET hive.optimize.index.filter=true;
+-- 4 mappers
+select userid,string1,subtype,decimal1,ts from orc_split_elim where userid<=29 order by userid;
+SET hive.optimize.index.filter=false;
+
+-- 5 mappers
+select userid,string1,subtype,decimal1,ts from orc_split_elim where userid<=70 order by userid;
+
+SET hive.optimize.index.filter=true;
+-- 5 mappers
+select userid,string1,subtype,decimal1,ts from orc_split_elim where userid<=70 order by userid;
+SET hive.optimize.index.filter=false;
+
+-- partitioned table
+create table orc_split_elim_part (userid bigint, string1 string, subtype double, decimal1 decimal, ts timestamp) partitioned by (country string, year int) stored as orc;
+
+alter table orc_split_elim_part add partition(country='us', year=2000);
+alter table orc_split_elim_part add partition(country='us', year=2001);
+
+load data local inpath '../../data/files/orc_split_elim.orc' into table orc_split_elim_part partition(country='us', year=2000);
+load data local inpath '../../data/files/orc_split_elim.orc' into table orc_split_elim_part partition(country='us', year=2001);
+
+-- 10 mapper - no split elimination
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=2 and country='us'order by userid;
+
+SET hive.optimize.index.filter=true;
+-- 2 mapper - split elimination
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=2 and country='us' order by userid;
+SET hive.optimize.index.filter=false;
+
+-- 10 mapper - no split elimination
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=2 and country='us' and (year=2000 or year=2001) order by userid;
+
+SET hive.optimize.index.filter=true;
+-- 2 mapper - split elimination
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=2 and country='us' and (year=2000 or year=2001) order by userid;
+SET hive.optimize.index.filter=false;
+
+-- 10 mapper - no split elimination
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=2 and country='us' and year=2000 order by userid;
+
+SET hive.optimize.index.filter=true;
+-- 1 mapper - split elimination
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=2 and country='us' and year=2000 order by userid;
+SET hive.optimize.index.filter=false;
+
+-- 10 mapper - no split elimination
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=5 and country='us' order by userid;
+
+SET hive.optimize.index.filter=true;
+-- 4 mapper - split elimination
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=5 and country='us' order by userid;
+SET hive.optimize.index.filter=false;
+
+-- 10 mapper - no split elimination
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=5 and country='us' and (year=2000 or year=2001) order by userid;
+
+SET hive.optimize.index.filter=true;
+-- 4 mapper - split elimination
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=5 and country='us' and (year=2000 or year=2001) order by userid;
+SET hive.optimize.index.filter=false;
+
+-- 10 mapper - no split elimination
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=5 and country='us' and year=2000 order by userid;
+
+SET hive.optimize.index.filter=true;
+-- 2 mapper - split elimination
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=5 and country='us' and year=2000 order by userid;
+SET hive.optimize.index.filter=false;
+
+-- 0 mapper - no split elimination
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=70 and country='in' order by userid;
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=70 and country='us' and year=2002 order by userid;
+
+SET hive.optimize.index.filter=true;
+-- 0 mapper - split elimination
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=70 and country='in' order by userid;
+select userid,string1,subtype,decimal1,ts from orc_split_elim_part where userid<=70 and country='us' and year=2002 order by userid;
+SET hive.optimize.index.filter=false;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_vectorization_ppd.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_vectorization_ppd.q
new file mode 100644
index 0000000000000000000000000000000000000000..9bdad86e41ebffb2bddbf176192991f28e0883d7
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_vectorization_ppd.q
@@ -0,0 +1,69 @@
+-- create table with 1000 rows
+create table srcorc(key string, value string) stored as textfile;
+insert overwrite table srcorc select * from src;
+insert into table srcorc select * from src;
+
+-- load table with each row group having 1000 rows and stripe 1 & 2 having 5000 & 2000 rows respectively
+create table if not exists vectororc
+(s1 string,
+s2 string,
+d double,
+s3 string)
+stored as ORC tblproperties("orc.row.index.stride"="1000", "orc.stripe.size"="100000", "orc.compress.size"="10000");
+
+-- insert creates separate orc files
+insert overwrite table vectororc select "apple", "a", rand(1), "zoo" from srcorc;
+insert into table vectororc select null, "b", rand(2), "zoo" from srcorc;
+insert into table vectororc select null, "c", rand(3), "zoo" from srcorc;
+insert into table vectororc select "apple", "d", rand(4), "zoo" from srcorc;
+insert into table vectororc select null, "e", rand(5), "z" from srcorc;
+insert into table vectororc select "apple", "f", rand(6), "z" from srcorc;
+insert into table vectororc select null, "g", rand(7), "zoo" from srcorc;
+
+-- since vectororc table has multiple orc file we will load them into a single file using another table
+create table if not exists testorc
+(s1 string,
+s2 string,
+d double,
+s3 string)
+stored as ORC tblproperties("orc.row.index.stride"="1000", "orc.stripe.size"="100000", "orc.compress.size"="10000");
+insert overwrite table testorc select * from vectororc order by s2;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+set hive.optimize.index.filter=true;
+
+set hive.vectorized.execution.enabled=false;
+-- row group (1,4) from stripe 1 and row group (1) from stripe 2
+-- PPD ONLY
+select count(*),int(sum(d)) from testorc where s1 is not null;
+set hive.vectorized.execution.enabled=true;
+-- VECTORIZATION + PPD
+select count(*),int(sum(d)) from testorc where s1 is not null;
+
+set hive.vectorized.execution.enabled=false;
+-- row group (2,3,5) from stripe 1 and row group (2) from stripe 2
+-- PPD ONLY
+select count(*),int(sum(d)) from testorc where s2 in ("b", "c", "e", "g");
+set hive.vectorized.execution.enabled=true;
+-- VECTORIZATION + PPD
+select count(*),int(sum(d)) from testorc where s2 in ("b", "c", "e", "g");
+
+set hive.vectorized.execution.enabled=false;
+-- last row group of stripe 1 and first row group of stripe 2
+-- PPD ONLY
+select count(*),int(sum(d)) from testorc where s3="z";
+set hive.vectorized.execution.enabled=true;
+-- VECTORIZATION + PPD
+select count(*),int(sum(d)) from testorc where s3="z";
+
+set hive.vectorized.execution.enabled=false;
+-- first row group of stripe 1 and last row group of stripe 2
+-- PPD ONLY
+select count(*),int(sum(d)) from testorc where s2="a" or s2="g";
+set hive.vectorized.execution.enabled=true;
+-- VECTORIZATION + PPD
+select count(*),int(sum(d)) from testorc where s2="a" or s2="g";
+
+drop table srcorc;
+drop table vectororc;
+drop table testorc;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/order_within_subquery.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/order_within_subquery.q
new file mode 100644
index 0000000000000000000000000000000000000000..7fc9b44cbc149c5c1479074b521c6ada141e617d
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/order_within_subquery.q
@@ -0,0 +1,19 @@
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+
+select t1.p_name, t2.p_name 
+from (select * from part order by p_size limit 10) t1 join part t2 on t1.p_partkey = t2.p_partkey and t1.p_size = t2.p_size 
+where t1.p_partkey < 100000;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q
index 5e09395901c08ceb30630263ca2d878b81be117b..73c39406448449bf8f7a75928cb5738e24aa49c6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q
@@ -1,6 +1,6 @@
 create table src5 (key string, value string);
-load data local inpath '../data/files/kv5.txt' into table src5;
-load data local inpath '../data/files/kv5.txt' into table src5;
+load data local inpath '../../data/files/kv5.txt' into table src5;
+load data local inpath '../../data/files/kv5.txt' into table src5;
 
 set mapred.reduce.tasks = 4;
 set hive.optimize.sampling.orderby=true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_create.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_create.q
new file mode 100644
index 0000000000000000000000000000000000000000..0b976bdbaf7004caa9ca669cf48bf4b84a6a6d46
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_create.q
@@ -0,0 +1,36 @@
+DROP TABLE parquet_create_staging;
+DROP TABLE parquet_create;
+
+CREATE TABLE parquet_create_staging (
+    id int,
+    str string,
+    mp  MAP<STRING,STRING>,
+    lst ARRAY<STRING>,
+    strct STRUCT<A:STRING,B:STRING>
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|'
+COLLECTION ITEMS TERMINATED BY ','
+MAP KEYS TERMINATED BY ':';
+
+CREATE TABLE parquet_create (
+    id int,
+    str string,
+    mp  MAP<STRING,STRING>,
+    lst ARRAY<STRING>,
+    strct STRUCT<A:STRING,B:STRING>
+) STORED AS PARQUET;
+
+DESCRIBE FORMATTED parquet_create;
+
+LOAD DATA LOCAL INPATH '../../data/files/parquet_create.txt' OVERWRITE INTO TABLE parquet_create_staging;
+
+SELECT * FROM parquet_create_staging;
+
+INSERT OVERWRITE TABLE parquet_create SELECT * FROM parquet_create_staging;
+
+SELECT * FROM parquet_create group by id;
+SELECT id, count(0) FROM parquet_create group by id;
+SELECT str from parquet_create;
+SELECT mp from parquet_create;
+SELECT lst from parquet_create;
+SELECT strct from parquet_create;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_ctas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_ctas.q
new file mode 100644
index 0000000000000000000000000000000000000000..652aef1b2ba533cb86887d741b4653f97f6d037c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_ctas.q
@@ -0,0 +1,24 @@
+drop table staging;
+drop table parquet_ctas;
+drop table parquet_ctas_advanced;
+drop table parquet_ctas_alias;
+drop table parquet_ctas_mixed;
+
+create table staging (key int, value string) stored as textfile;
+insert into table staging select * from src order by key limit 10;
+
+create table parquet_ctas stored as parquet as select * from staging;
+describe parquet_ctas;
+select * from parquet_ctas;
+
+create table parquet_ctas_advanced stored as parquet as select key+1,concat(value,"value") from staging;
+describe parquet_ctas_advanced;
+select * from parquet_ctas_advanced;
+
+create table parquet_ctas_alias stored as parquet as select key+1 as mykey,concat(value,"value") as myvalue from staging;
+describe parquet_ctas_alias;
+select * from parquet_ctas_alias;
+
+create table parquet_ctas_mixed stored as parquet as select key,key+1,concat(value,"value") as myvalue from staging;
+describe parquet_ctas_mixed;
+select * from parquet_ctas_mixed;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_partitioned.q
new file mode 100644
index 0000000000000000000000000000000000000000..5d4f68ea4372319a3ef40f91e63be6867526d086
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_partitioned.q
@@ -0,0 +1,34 @@
+set hive.exec.dynamic.partition.mode=nonstrict;
+set hive.exec.dynamic.partition=true;
+
+DROP TABLE parquet_partitioned_staging;
+DROP TABLE parquet_partitioned;
+
+CREATE TABLE parquet_partitioned_staging (
+    id int,
+    str string,
+    part string
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|';
+
+CREATE TABLE parquet_partitioned (
+    id int,
+    str string
+) PARTITIONED BY (part string)
+STORED AS PARQUET;
+
+DESCRIBE FORMATTED parquet_partitioned;
+
+LOAD DATA LOCAL INPATH '../../data/files/parquet_partitioned.txt' OVERWRITE INTO TABLE parquet_partitioned_staging;
+
+SELECT * FROM parquet_partitioned_staging;
+
+INSERT OVERWRITE TABLE parquet_partitioned PARTITION (part) SELECT * FROM parquet_partitioned_staging;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+SELECT * FROM parquet_partitioned ORDER BY id, str;
+SELECT part, COUNT(0) FROM parquet_partitioned GROUP BY part;
+
+set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+SELECT * FROM parquet_partitioned ORDER BY id, str;
+SELECT part, COUNT(0) FROM parquet_partitioned GROUP BY part;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_types.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_types.q
new file mode 100644
index 0000000000000000000000000000000000000000..5d6333c934b74c0615503b182cf5161d134f14f5
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_types.q
@@ -0,0 +1,38 @@
+DROP TABLE parquet_types_staging;
+DROP TABLE parquet_types;
+
+CREATE TABLE parquet_types_staging (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string
+) ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|';
+
+CREATE TABLE parquet_types (
+  cint int,
+  ctinyint tinyint,
+  csmallint smallint,
+  cfloat float,
+  cdouble double,
+  cstring1 string
+) STORED AS PARQUET;
+
+LOAD DATA LOCAL INPATH '../../data/files/parquet_types.txt' OVERWRITE INTO TABLE parquet_types_staging;
+
+INSERT OVERWRITE TABLE parquet_types SELECT * FROM parquet_types_staging;
+
+SELECT * FROM parquet_types;
+
+SELECT ctinyint,
+  MAX(cint),
+  MIN(csmallint),
+  COUNT(cstring1),
+  AVG(cfloat),
+  STDDEV_POP(cdouble)
+FROM parquet_types
+GROUP BY ctinyint
+ORDER BY ctinyint
+;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q
index b7f8c64d4261fba4004700a474d48a9d18733798..03a5760e690e9d2a469497971c084267d3764ab2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q
@@ -1,7 +1,7 @@
 
 create table test1(col1 string) partitioned by (partitionId int);
 insert overwrite table test1 partition (partitionId=1)
-  select key from src limit 10;
+  select key from src tablesample (10 rows);
 
  FROM (
  FROM test1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q
index 8738afdfa099aec6882ef5348f4b107c429986ca..70a7b252154d141c20bde77d3596877f237df913 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q
@@ -1,45 +1,58 @@
 drop table partition_date_1;
 
-create table partition_date_1 (key string, value string) partitioned by (dt date, region int);
+create table partition_date_1 (key string, value string) partitioned by (dt date, region string);
+
+insert overwrite table partition_date_1 partition(dt='2000-01-01', region= '1')
+  select * from src tablesample (10 rows);
+insert overwrite table partition_date_1 partition(dt='2000-01-01', region= '2')
+  select * from src tablesample (5 rows);
+insert overwrite table partition_date_1 partition(dt='2013-12-10', region= '2020-20-20')
+  select * from src tablesample (5 rows);
+insert overwrite table partition_date_1 partition(dt='2013-08-08', region= '1') 
+  select * from src tablesample (20 rows);
+insert overwrite table partition_date_1 partition(dt='2013-08-08', region= '10') 
+  select * from src tablesample (11 rows);
 
-insert overwrite table partition_date_1 partition(dt='2000-01-01', region=1) 
-  select * from src limit 10;
-insert overwrite table partition_date_1 partition(dt='2000-01-01', region=2) 
-  select * from src limit 5;
-insert overwrite table partition_date_1 partition(dt='2013-08-08', region=1) 
-  select * from src limit 20;
-insert overwrite table partition_date_1 partition(dt='2013-08-08', region=10) 
-  select * from src limit 11;
 
 select distinct dt from partition_date_1;
-select * from partition_date_1 where dt = '2000-01-01' and region = 2 order by key,value;
+select * from partition_date_1 where dt = '2000-01-01' and region = '2' order by key,value;
 
 -- 15
 select count(*) from partition_date_1 where dt = date '2000-01-01';
 -- 15.  Also try with string value in predicate
 select count(*) from partition_date_1 where dt = '2000-01-01';
 -- 5
-select count(*) from partition_date_1 where dt = date '2000-01-01' and region = 2;
+select count(*) from partition_date_1 where dt = date '2000-01-01' and region = '2';
 -- 11
-select count(*) from partition_date_1 where dt = date '2013-08-08' and region = 10;
+select count(*) from partition_date_1 where dt = date '2013-08-08' and region = '10';
 -- 30
-select count(*) from partition_date_1 where region = 1;
+select count(*) from partition_date_1 where region = '1';
 -- 0
-select count(*) from partition_date_1 where dt = date '2000-01-01' and region = 3;
+select count(*) from partition_date_1 where dt = date '2000-01-01' and region = '3';
 -- 0
 select count(*) from partition_date_1 where dt = date '1999-01-01';
 
 -- Try other comparison operations
 
 -- 20
-select count(*) from partition_date_1 where dt > date '2000-01-01' and region = 1;
+select count(*) from partition_date_1 where dt > date '2000-01-01' and region = '1';
 -- 10
-select count(*) from partition_date_1 where dt < date '2000-01-02' and region = 1;
+select count(*) from partition_date_1 where dt < date '2000-01-02' and region = '1';
 -- 20
-select count(*) from partition_date_1 where dt >= date '2000-01-02' and region = 1;
+select count(*) from partition_date_1 where dt >= date '2000-01-02' and region = '1';
 -- 10
-select count(*) from partition_date_1 where dt <= date '2000-01-01' and region = 1;
+select count(*) from partition_date_1 where dt <= date '2000-01-01' and region = '1';
 -- 20
-select count(*) from partition_date_1 where dt <> date '2000-01-01' and region = 1;
+select count(*) from partition_date_1 where dt <> date '2000-01-01' and region = '1';
+-- 10
+select count(*) from partition_date_1 where dt between date '1999-12-30' and date '2000-01-03' and region = '1';
+
+
+-- Try a string key with date-like strings
+
+-- 5
+select count(*) from partition_date_1 where region = '2020-20-20';
+-- 5
+select count(*) from partition_date_1 where region > '2010-01-01';
 
 drop table partition_date_1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q
index 9b84b5960850396536d7934cb0d166002b15027e..c932ed102363726ca5321591ed0f8c2083d01c6d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q
@@ -3,7 +3,7 @@ drop table partition_date2_1;
 create table partition_date2_1 (key string, value string) partitioned by (dt date, region int);
 
 -- test date literal syntax
-from (select * from src limit 1) x
+from (select * from src tablesample (1 rows)) x
 insert overwrite table partition_date2_1 partition(dt=date '2000-01-01', region=1) select *
 insert overwrite table partition_date2_1 partition(dt=date '2000-01-01', region=2) select *
 insert overwrite table partition_date2_1 partition(dt=date '1999-01-01', region=2) select *;
@@ -13,7 +13,7 @@ select * from partition_date2_1;
 
 -- insert overwrite
 insert overwrite table partition_date2_1 partition(dt=date '2000-01-01', region=2) 
-  select 'changed_key', 'changed_value' from src limit 2;
+  select 'changed_key', 'changed_value' from src tablesample (2 rows);
 select * from partition_date2_1;
 
 -- truncate
@@ -41,7 +41,7 @@ alter table partition_date2_1 partition(dt=date '1980-01-02', region=3)
 describe extended partition_date2_1  partition(dt=date '1980-01-02', region=3);
 
 insert overwrite table partition_date2_1 partition(dt=date '1980-01-02', region=3)
-  select * from src limit 2;
+  select * from src tablesample (2 rows);
 select * from partition_date2_1 order by key,value,dt,region;
 
 -- alter table set location
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q
index ba193cd51a26d73e7b1f450088c4450e69562bc8..a8381a4200f0a26a227fd156af3f9b7e38dd7a21 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q
@@ -1,9 +1,9 @@
 create table sc as select * 
-from (select '2011-01-11', '2011-01-11+14:18:26' from src limit 1 
+from (select '2011-01-11', '2011-01-11+14:18:26' from src tablesample (1 rows)
       union all 
-      select '2011-01-11', '2011-01-11+15:18:26' from src limit 1 
+      select '2011-01-11', '2011-01-11+15:18:26' from src tablesample (1 rows)
       union all 
-      select '2011-01-11', '2011-01-11+16:18:26' from src limit 1 ) s;
+      select '2011-01-11', '2011-01-11+16:18:26' from src tablesample (1 rows) ) s;
 
 create table sc_part (key string) partitioned by (ts string) stored as rcfile;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q
index 81344334dfe0156a43f308ac69d90f0ad132b1b9..b0b1ff4db607493dc8d4535b3b79399475b01b70 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q
@@ -1,9 +1,9 @@
 create table sc as select * 
-from (select '2011-01-11', '2011-01-11+14:18:26' from src limit 1 
+from (select '2011-01-11', '2011-01-11+14:18:26' from src tablesample (1 rows)
       union all 
-      select '2011-01-11', '2011-01-11+15:18:26' from src limit 1 
+      select '2011-01-11', '2011-01-11+15:18:26' from src tablesample (1 rows)
       union all 
-      select '2011-01-11', '2011-01-11+16:18:26' from src limit 1 ) s;
+      select '2011-01-11', '2011-01-11+16:18:26' from src tablesample (1 rows) ) s;
 
 create table sc_part (key string) partitioned by (ts string) stored as rcfile;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q
index 7f1accadac6ea771cc022a14d5b07844fcba9908..c9bca99b9cdf679665ea223034b30ff0a405827d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q
@@ -2,14 +2,14 @@ set hive.typecheck.on.insert = true;
 
 -- begin part(string, string) pass(string, int)
 CREATE TABLE tab1 (id1 int,id2 string) PARTITIONED BY(month string,day string) stored as textfile;
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day=2);
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day=2);
 
 select * from tab1;
 drop table tab1;
 
 -- begin part(string, int) pass(string, string)
 CREATE TABLE tab1 (id1 int,id2 string) PARTITIONED BY(month string,day int) stored as textfile;
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='2');
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='2');
 
 select * from tab1;
 drop table tab1;
@@ -17,7 +17,7 @@ drop table tab1;
 -- begin part(string, date) pass(string, date)
 create table tab1 (id1 int, id2 string) PARTITIONED BY(month string,day date) stored as textfile;
 alter table tab1 add partition (month='June', day='2008-01-01');
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='2008-01-01');
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='2008-01-01');
 
 select id1, id2, day from tab1 where day='2008-01-01';
 drop table tab1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q
index d700b1cbf8566060adb2f617f5b7717a421063d8..22aadd3b5359fa7f094561586b2768e2520bec55 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q
@@ -3,13 +3,13 @@ drop table partition_varchar_1;
 create table partition_varchar_1 (key string, value varchar(20)) partitioned by (dt varchar(10), region int);
 
 insert overwrite table partition_varchar_1 partition(dt='2000-01-01', region=1)
-  select * from src limit 10;
+  select * from src tablesample (10 rows);
 insert overwrite table partition_varchar_1 partition(dt='2000-01-01', region=2)
-  select * from src limit 5;
+  select * from src tablesample (5 rows);
 insert overwrite table partition_varchar_1 partition(dt='2013-08-08', region=1)
-  select * from src limit 20;
+  select * from src tablesample (20 rows);
 insert overwrite table partition_varchar_1 partition(dt='2013-08-08', region=10)
-  select * from src limit 11;
+  select * from src tablesample (11 rows);
 
 select distinct dt from partition_varchar_1;
 select * from partition_varchar_1 where dt = '2000-01-01' and region = 2 order by key,value;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar2.q
new file mode 100644
index 0000000000000000000000000000000000000000..92cb742f150115e75d3dff03d1d325c38ccd8357
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar2.q
@@ -0,0 +1,10 @@
+drop table partition_varchar_2;
+
+create table partition_varchar_2 (key string, value varchar(20)) partitioned by (dt varchar(15), region int);
+
+insert overwrite table partition_varchar_2 partition(dt='2000-01-01', region=1)
+  select * from src order by key limit 1;
+
+select * from partition_varchar_2 where cast(dt as varchar(10)) = '2000-01-01';
+
+drop table partition_varchar_2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q
index e9b574c1ca380ddb5dd8adc984a3c262952095e2..3cf488fb0337fcba95a937f0faa6ac52e309164a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q
@@ -3,9 +3,9 @@
 -- CustomSerDe(1, 2, 3) irrespective of the inserted values
 
 DROP TABLE PW17;
-ADD JAR ../build/ql/test/test-serdes.jar;
+ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-custom-serde/${system:hive.version}/hive-it-custom-serde-${system:hive.version}.jar;
 CREATE TABLE PW17(USER STRING, COMPLEXDT ARRAY<INT>) PARTITIONED BY (YEAR STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe1';
-LOAD DATA LOCAL INPATH '../data/files/pw17.txt' INTO TABLE PW17 PARTITION (YEAR='1');
+LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW17 PARTITION (YEAR='1');
 ALTER TABLE PW17 PARTITION(YEAR='1') SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe2';
 ALTER TABLE PW17 SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe1';
 -- Without the fix HIVE-5199, will throw cast exception via FetchOperator
@@ -14,13 +14,13 @@ SELECT * FROM PW17;
 -- Test for non-parititioned table. 
 DROP TABLE PW17_2;
 CREATE TABLE PW17_2(USER STRING, COMPLEXDT ARRAY<INT>) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe1';
-LOAD DATA LOCAL INPATH '../data/files/pw17.txt' INTO TABLE PW17_2;
+LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW17_2;
 -- Without the fix HIVE-5199, will throw cast exception via MapOperator
 SELECT COUNT(*) FROM PW17_2;
 
 DROP TABLE PW17_3;
 CREATE TABLE PW17_3(USER STRING, COMPLEXDT ARRAY<ARRAY<INT> >) PARTITIONED BY (YEAR STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe3';
-LOAD DATA LOCAL INPATH '../data/files/pw17.txt' INTO TABLE PW17_3 PARTITION (YEAR='1');
+LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW17_3 PARTITION (YEAR='1');
 ALTER TABLE PW17_3 PARTITION(YEAR='1') SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe2';
 ALTER TABLE PW17_3 SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe3';
 -- Without the fix HIVE-5285, will throw cast exception via FetchOperator
@@ -28,7 +28,7 @@ SELECT * FROM PW17;
 
 DROP TABLE PW17_4;
 CREATE TABLE PW17_4(USER STRING, COMPLEXDT ARRAY<ARRAY<INT> >) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe3';
-LOAD DATA LOCAL INPATH '../data/files/pw17.txt' INTO TABLE PW17_4;
+LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW17_4;
 -- Without the fix HIVE-5285, will throw cast exception via MapOperator
 SELECT COUNT(*) FROM PW17_4;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat18.q
new file mode 100644
index 0000000000000000000000000000000000000000..40ed2585f5122a8e7080e476bbe90e70e099f3ae
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat18.q
@@ -0,0 +1,19 @@
+-- HIVE-5202 : Tests for SettableUnionObjectInspectors
+-- CustomSerDe(4,5) are used here. 
+-- The final results should be all NULL columns deserialized using 
+-- CustomSerDe(4, 5) irrespective of the inserted values
+
+DROP TABLE PW18;
+ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-custom-serde/${system:hive.version}/hive-it-custom-serde-${system:hive.version}.jar;
+CREATE TABLE PW18(USER STRING, COMPLEXDT UNIONTYPE<INT, DOUBLE>) PARTITIONED BY (YEAR STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe5';
+LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW18 PARTITION (YEAR='1');
+ALTER TABLE PW18 PARTITION(YEAR='1') SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe4';
+-- Without the fix HIVE-5202, will throw unsupported data type exception.
+SELECT * FROM PW18;
+
+-- Test for non-parititioned table. 
+DROP TABLE PW18_2;
+CREATE TABLE PW18_2(USER STRING, COMPLEXDT UNIONTYPE<INT, DOUBLE>) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe5';
+LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW18_2;
+-- Without the fix HIVE-5202, will throw unsupported data type exception
+SELECT COUNT(*) FROM PW18_2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q
index 09a39ae4e4476a0abd5519a90c16fb71f34325aa..3be0ff23b8124c2bbfcdfaa55b28bc58e79f2f41 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q
@@ -127,7 +127,7 @@ create table ab(strct struct<a:int, b:string>)
 row format delimited
   fields terminated by '\t'
   collection items terminated by '\001';
-load data local inpath '../data/files/kv1.txt'
+load data local inpath '../../data/files/kv1.txt'
 overwrite into table ab;
 
 -- Create partitioned table with struct data:
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join4.q
new file mode 100644
index 0000000000000000000000000000000000000000..475d45c19ff0c74c8a4d4706e43e09d296d91f19
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join4.q
@@ -0,0 +1,22 @@
+create table dual(a string);
+
+set hive.optimize.ppd=true;
+drop table if exists test_tbl ;
+
+create table test_tbl (id string,name string);
+
+insert into table test_tbl
+select 'a','b' from dual;
+
+explain
+select t2.* 
+from
+(select id,name from (select id,name from test_tbl) t1 sort by id) t2
+join test_tbl t3 on (t2.id=t3.id )
+where t2.name='c' and t3.id='a';
+
+select t2.* 
+from
+(select id,name from (select id,name from test_tbl) t1 sort by id) t2
+join test_tbl t3 on (t2.id=t3.id )
+where t2.name='c' and t3.id='a';
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q
index a802df1b98819475210f612319bbcde57aeb15ae..06fe7ce580c84733f0b62bac795c91e23cf0eb5f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q
@@ -10,18 +10,18 @@ FROM src a JOIN src b ON (a.key = b.key)
 INSERT OVERWRITE TABLE mi1 SELECT a.* WHERE a.key < 100
 INSERT OVERWRITE TABLE mi2 SELECT a.key, a.value WHERE a.key >= 100 and a.key < 200
 INSERT OVERWRITE TABLE mi3 PARTITION(ds='2008-04-08', hr='12') SELECT a.key WHERE a.key >= 200 and a.key < 300
-INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300;
+INSERT OVERWRITE DIRECTORY 'target/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300;
 
 FROM src a JOIN src b ON (a.key = b.key)
 INSERT OVERWRITE TABLE mi1 SELECT a.* WHERE a.key < 100
 INSERT OVERWRITE TABLE mi2 SELECT a.key, a.value WHERE a.key >= 100 and a.key < 200
 INSERT OVERWRITE TABLE mi3 PARTITION(ds='2008-04-08', hr='12') SELECT a.key WHERE a.key >= 200 and a.key < 300
-INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300;
+INSERT OVERWRITE DIRECTORY 'target/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300;
 
 SELECT mi1.* FROM mi1;
 SELECT mi2.* FROM mi2;
 SELECT mi3.* FROM mi3;
-dfs -cat ../build/ql/test/data/warehouse/mi4.out/*;
+dfs -cat ${system:test.warehouse.dir}/mi4.out/*;
 
 
 set hive.ppd.remove.duplicatefilters=true;
@@ -31,15 +31,15 @@ FROM src a JOIN src b ON (a.key = b.key)
 INSERT OVERWRITE TABLE mi1 SELECT a.* WHERE a.key < 100
 INSERT OVERWRITE TABLE mi2 SELECT a.key, a.value WHERE a.key >= 100 and a.key < 200
 INSERT OVERWRITE TABLE mi3 PARTITION(ds='2008-04-08', hr='12') SELECT a.key WHERE a.key >= 200 and a.key < 300
-INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300;
+INSERT OVERWRITE DIRECTORY 'target/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300;
 
 FROM src a JOIN src b ON (a.key = b.key)
 INSERT OVERWRITE TABLE mi1 SELECT a.* WHERE a.key < 100
 INSERT OVERWRITE TABLE mi2 SELECT a.key, a.value WHERE a.key >= 100 and a.key < 200
 INSERT OVERWRITE TABLE mi3 PARTITION(ds='2008-04-08', hr='12') SELECT a.key WHERE a.key >= 200 and a.key < 300
-INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300;
+INSERT OVERWRITE DIRECTORY 'target/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300;
 
 SELECT mi1.* FROM mi1;
 SELECT mi2.* FROM mi2;
 SELECT mi3.* FROM mi3;
-dfs -cat ../build/ql/test/data/warehouse/mi4.out/*;
+dfs -cat ${system:test.warehouse.dir}/mi4.out/*;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q
index 65a498d021f77ae26d195b3974f7ae5abe261a15..530ef9c4d849a3739cca8c52f1e21a1369327784 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q
@@ -36,3 +36,12 @@ FROM (
   CLUSTER BY tkey 
 ) tmap
 SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100;
+
+-- test described in HIVE-4598
+
+EXPLAIN
+FROM (
+    FROM ( SELECT * FROM src ) mapout REDUCE * USING 'cat' AS x,y
+) reduced
+insert overwrite local directory '/tmp/a' select * where x='a' or x='b'
+insert overwrite local directory '/tmp/b' select * where x='c' or x='d';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udtf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udtf.q
new file mode 100644
index 0000000000000000000000000000000000000000..d90532cfa4bcd05c029a737fe826be0c51836eb4
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udtf.q
@@ -0,0 +1,12 @@
+explain
+SELECT value from (
+  select explode(array(key, value)) as (value) from (
+    select * FROM src WHERE key > 400
+  ) A
+) B WHERE value < 450;
+
+SELECT value from (
+  select explode(array(key, value)) as (value) from (
+    select * FROM src WHERE key > 400
+  ) A
+) B WHERE value < 450;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q
index d635e2d171469ec930d97aef33c08adbe10af9f6..a7606c5a5f2a506b9fd91fd360a5c754db9a3bc2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q
@@ -5,26 +5,26 @@ drop view v;
 create table t1_new (key string, value string) partitioned by (ds string);
 
 insert overwrite table t1_new partition (ds = '2011-10-15')
-select 'key1', 'value1' from src limit 1;
+select 'key1', 'value1' from src tablesample (1 rows);
 
 insert overwrite table t1_new partition (ds = '2011-10-16')
-select 'key2', 'value2' from src limit 1;
+select 'key2', 'value2' from src tablesample (1 rows);
 
 create table t1_old (keymap string, value string) partitioned by (ds string);
 
 insert overwrite table t1_old partition (ds = '2011-10-13')
-select 'keymap3', 'value3' from src limit 1;
+select 'keymap3', 'value3' from src tablesample (1 rows);
 
 insert overwrite table t1_old partition (ds = '2011-10-14')
-select 'keymap4', 'value4' from src limit 1;
+select 'keymap4', 'value4' from src tablesample (1 rows);
 
 create table t1_mapping (key string, keymap string) partitioned by (ds string);
 
 insert overwrite table t1_mapping partition (ds = '2011-10-13')
-select 'key3', 'keymap3' from src limit 1;
+select 'key3', 'keymap3' from src tablesample (1 rows);
 
 insert overwrite table t1_mapping partition (ds = '2011-10-14')
-select 'key4', 'keymap4' from src limit 1;
+select 'key4', 'keymap4' from src tablesample (1 rows);
 
 
 create view t1 partitioned on (ds) as
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q
index 860dd631ce10aaafdd67b8363a7a44688eedf027..440005fdee951b13c407b08dd5dc796c092d3dad 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 create table ppr_test (key string) partitioned by (ds string);
 
 alter table ppr_test add partition (ds = '1234');
@@ -9,14 +11,14 @@ alter table ppr_test add partition (ds = '12:4');
 alter table ppr_test add partition (ds = '12%4');
 alter table ppr_test add partition (ds = '12*4');
 
-insert overwrite table ppr_test partition(ds = '1234') select * from (select '1234' from src limit 1 union all select 'abcd' from src limit 1) s;
-insert overwrite table ppr_test partition(ds = '1224') select * from (select '1224' from src limit 1 union all select 'abcd' from src limit 1) s;
-insert overwrite table ppr_test partition(ds = '1214') select * from (select '1214' from src limit 1 union all select 'abcd' from src limit 1) s;
-insert overwrite table ppr_test partition(ds = '12+4') select * from (select '12+4' from src limit 1 union all select 'abcd' from src limit 1) s;
-insert overwrite table ppr_test partition(ds = '12.4') select * from (select '12.4' from src limit 1 union all select 'abcd' from src limit 1) s;
-insert overwrite table ppr_test partition(ds = '12:4') select * from (select '12:4' from src limit 1 union all select 'abcd' from src limit 1) s;
-insert overwrite table ppr_test partition(ds = '12%4') select * from (select '12%4' from src limit 1 union all select 'abcd' from src limit 1) s;
-insert overwrite table ppr_test partition(ds = '12*4') select * from (select '12*4' from src limit 1 union all select 'abcd' from src limit 1) s;
+insert overwrite table ppr_test partition(ds = '1234') select * from (select '1234' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s;
+insert overwrite table ppr_test partition(ds = '1224') select * from (select '1224' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s;
+insert overwrite table ppr_test partition(ds = '1214') select * from (select '1214' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s;
+insert overwrite table ppr_test partition(ds = '12+4') select * from (select '12+4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s;
+insert overwrite table ppr_test partition(ds = '12.4') select * from (select '12.4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s;
+insert overwrite table ppr_test partition(ds = '12:4') select * from (select '12:4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s;
+insert overwrite table ppr_test partition(ds = '12%4') select * from (select '12%4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s;
+insert overwrite table ppr_test partition(ds = '12*4') select * from (select '12*4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s;
 
 
 select * from ppr_test where ds = '1234' order by key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q
index 67c0da0dfc59126f6a74603ead0d28f209f4536f..8c6090653811beb695011780b9ca2daf5c6e7694 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q
@@ -1,24 +1,26 @@
+set hive.fetch.task.conversion=more;
+
 create table ppr_test (key string) partitioned by (ds string);
 
-insert overwrite table ppr_test partition(ds='2') select '2' from src limit 1;
-insert overwrite table ppr_test partition(ds='22') select '22' from src limit 1;
+insert overwrite table ppr_test partition(ds='2') select '2' from src tablesample (1 rows);
+insert overwrite table ppr_test partition(ds='22') select '22' from src tablesample (1 rows);
 
 select * from ppr_test where ds = '2';
 select * from ppr_test where ds = '22';
 
 
 create table ppr_test2 (key string) partitioned by (ds string, s string);
-insert overwrite table ppr_test2 partition(ds='1', s='2') select '1' from src limit 1;
-insert overwrite table ppr_test2 partition(ds='2', s='1') select '2' from src limit 1;
+insert overwrite table ppr_test2 partition(ds='1', s='2') select '1' from src tablesample (1 rows);
+insert overwrite table ppr_test2 partition(ds='2', s='1') select '2' from src tablesample (1 rows);
 
 select * from ppr_test2 where s = '1';
 select * from ppr_test2 where ds = '1';
 
 
 create table ppr_test3 (key string) partitioned by (col string, ol string, l string);
-insert overwrite table ppr_test3 partition(col='1', ol='2', l = '3') select '1' from src limit 1;
-insert overwrite table ppr_test3 partition(col='1', ol='1', l = '2') select '2' from src limit 1;
-insert overwrite table ppr_test3 partition(col='1', ol='2', l = '1') select '3' from src limit 1;
+insert overwrite table ppr_test3 partition(col='1', ol='2', l = '3') select '1' from src tablesample (1 rows);
+insert overwrite table ppr_test3 partition(col='1', ol='1', l = '2') select '2' from src tablesample (1 rows);
+insert overwrite table ppr_test3 partition(col='1', ol='2', l = '1') select '3' from src tablesample (1 rows);
 
 select * from ppr_test3 where l = '1';
 select * from ppr_test3 where l = '2';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q
index ad908a02ad1815d28354156d7f677d6b0aebb517..22ee92634d123e5317a368f9b52b6ff234fab52f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q
@@ -2,7 +2,7 @@ set hive.heartbeat.interval=5;
 
 
 CREATE TABLE PROGRESS_1(key int, value string) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv6.txt' INTO TABLE PROGRESS_1;
+LOAD DATA LOCAL INPATH '../../data/files/kv6.txt' INTO TABLE PROGRESS_1;
 
 select count(1) from PROGRESS_1 t1 join PROGRESS_1 t2 on t1.key=t2.key;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q
index eea5415d682a16416b658747f6a46663520c57ef..d56b4123554c40089c5d56bc2f007622a652e25b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q
@@ -13,7 +13,7 @@ CREATE TABLE part(
     p_comment STRING
 );
 
-LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part;
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
 
 --1. test1
 select p_mfgr, p_name, p_size,
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q
index 03f435e4539e20791212ebaf9af5d2696ae7728a..9799534ff4a55287ed6218f06e3dde1e2e266c46 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q
@@ -9,11 +9,11 @@ CREATE TABLE part(
     p_type STRING,
     p_size INT,
     p_container STRING,
-    p_retailprice DECIMAL,
+    p_retailprice DECIMAL(6,2),
     p_comment STRING
 );
 
-LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part;
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
 
 -- 1. aggregate functions with decimal type
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q
index 885c3b3d43e17c117a0e9cf085dfba551b7b33b0..4fe9710d0d247829bc4694cfbac62832b1d578d7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q
@@ -13,7 +13,7 @@ CREATE TABLE part(
     p_comment STRING
 );
 
-LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part;
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
 
 -- 1. testNoPTFNoWindowing
 select p_mfgr, p_name, p_size
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q
index 72eeb104d529815f629cc69db9509544a3fb1bfc..0cde350f736939a87fb87748be047de93b5ca579 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q
@@ -10,7 +10,7 @@ ARR_DELAY float,
 FL_NUM string 
 );
 
-LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny;
+LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny;
 
 -- 1. basic Matchpath test
 select origin_city_name, fl_num, year, month, day_of_month, sz, tpath 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q
index 535a233a9e3c43f17851c2fb5f7d9754b19ce83f..a68c578848dcb1b4382a72ea9270568520bd4eca 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q
@@ -12,7 +12,7 @@ CREATE TABLE part_rc(
     p_comment STRING
 )  STORED AS RCFILE ;
 
-LOAD DATA LOCAL INPATH '../data/files/part.rc' overwrite into table part_rc;
+LOAD DATA LOCAL INPATH '../../data/files/part.rc' overwrite into table part_rc;
 
 -- testWindowingPTFWithPartRC
 select p_mfgr, p_name, p_size, 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q
index a2140cd049f2dd36421e99a7e3e3c932a28187b5..4b508e9df0449fa8404d1f8b829508518a41e5ec 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q
@@ -10,7 +10,7 @@ ARR_DELAY float,
 FL_NUM string 
 );
 
-LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny;
+LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny;
 
 create temporary function matchpathtest as 'org.apache.hadoop.hive.ql.udf.ptf.MatchPath$MatchPathResolver';
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q
index 4aa8ce11bec9c96d11eeaaf928f7cd13a428d2b9..c5d65f0efa211cdb798b3b2d0c5b1c15aa2c02bc 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q
@@ -12,7 +12,7 @@ CREATE TABLE part_seq(
     p_comment STRING
 ) STORED AS SEQUENCEFILE ;
 
-LOAD DATA LOCAL INPATH '../data/files/part.seq' overwrite into table part_seq;
+LOAD DATA LOCAL INPATH '../../data/files/part.seq' overwrite into table part_seq;
 
 -- testWindowingPTFWithPartSeqFile
 select p_mfgr, p_name, p_size, 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q
index f198baa6e4c5bf982bb04d5643694c1b45b9d47d..57e8cc673caced26425476b646367a84e41a14dd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q
@@ -1,3 +1,5 @@
+set hive.stats.dbclass=counter;
+set hive.stats.autogather=true;
 
 DROP TABLE lineitem;
 CREATE TABLE lineitem (L_ORDERKEY      INT,
@@ -19,7 +21,7 @@ CREATE TABLE lineitem (L_ORDERKEY      INT,
 ROW FORMAT DELIMITED
 FIELDS TERMINATED BY '|';
 
-LOAD DATA LOCAL INPATH '../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem;
+LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem;
 
 CREATE INDEX lineitem_lshipdate_idx ON TABLE lineitem(l_shipdate) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(l_shipdate)");
 ALTER INDEX lineitem_lshipdate_idx ON lineitem REBUILD;
@@ -156,7 +158,7 @@ DROP INDEX tbl_part_index on tblpart;
 DROP TABLE tblpart;
 
 CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'; 
-LOAD DATA LOCAL INPATH '../data/files/tbl.txt' OVERWRITE INTO TABLE tbl;
+LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE INTO TABLE tbl;
 
 CREATE INDEX tbl_key_idx ON TABLE tbl(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)");
 ALTER INDEX tbl_key_idx ON tbl REBUILD;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q
index 65b9f8776d192721b378e7d0890b06cbc65153e9..c93902ab3e39ca8c94a080fe8fc1deccd07665c8 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 EXPLAIN
 SELECT
     'abc',        "abc",
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_alter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_alter.q
new file mode 100644
index 0000000000000000000000000000000000000000..a34a25af4bb962208e2d5c14b2525c24547856ef
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_alter.q
@@ -0,0 +1,21 @@
+
+set hive.support.quoted.identifiers=column;
+
+create table src_b3(`x+1` string, `!@#$%^&*()_q` string) ;
+
+alter table src_b3 
+clustered by (`!@#$%^&*()_q`) sorted by (`!@#$%^&*()_q`) into 2 buckets
+;
+
+
+-- alter partition
+create table src_p3(`x+1` string, `y&y` string) partitioned by (`!@#$%^&*()_q` string);
+
+insert overwrite table src_p3 partition(`!@#$%^&*()_q`='a') select * from src;
+show partitions src_p3;
+
+alter table src_p3 add if not exists partition(`!@#$%^&*()_q`='b');
+show partitions src_p3;
+
+alter table src_p3 partition(`!@#$%^&*()_q`='b') rename to partition(`!@#$%^&*()_q`='c');
+show partitions src_p3;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_basic.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_basic.q
new file mode 100644
index 0000000000000000000000000000000000000000..680868e549ceee88ff2977ddda369a72ab7edbc9
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_basic.q
@@ -0,0 +1,34 @@
+
+set hive.support.quoted.identifiers=column;
+
+-- basic
+create table t1(`x+1` string, `y&y` string, `!@#$%^&*()_q` string);
+describe t1;
+select `x+1`, `y&y`, `!@#$%^&*()_q` from t1;
+explain select `x+1`, `y&y`, `!@#$%^&*()_q` from t1;
+explain select `x+1`, `y&y`, `!@#$%^&*()_q` from t1 where `!@#$%^&*()_q` = '1';
+explain select `x+1`, `y&y`, `!@#$%^&*()_q` from t1 where `!@#$%^&*()_q` = '1' group by `x+1`, `y&y`, `!@#$%^&*()_q` having `!@#$%^&*()_q` = '1';
+explain select `x+1`, `y&y`, `!@#$%^&*()_q`, rank() over(partition by `!@#$%^&*()_q` order by  `y&y`)  
+from t1 where `!@#$%^&*()_q` = '1' group by `x+1`, `y&y`, `!@#$%^&*()_q` having `!@#$%^&*()_q` = '1';
+
+-- case insensitive
+explain select `X+1`, `Y&y`, `!@#$%^&*()_Q`, rank() over(partition by `!@#$%^&*()_q` order by  `y&y`)  
+from t1 where `!@#$%^&*()_q` = '1' group by `x+1`, `y&Y`, `!@#$%^&*()_q` having `!@#$%^&*()_Q` = '1';
+
+
+-- escaped back ticks
+create table t4(`x+1``` string, `y&y` string);
+describe t4;
+insert into table t4 select * from src;
+select `x+1```, `y&y`, rank() over(partition by `x+1``` order by  `y&y`)  
+from t4 where `x+1``` = '10' group by `x+1```, `y&y` having `x+1``` = '10';
+
+-- view
+create view v1 as 
+select `x+1```, `y&y`
+from t4 where `x+1``` < '200';
+
+select `x+1```, `y&y`, rank() over(partition by `x+1``` order by  `y&y`)
+from v1
+group by `x+1```, `y&y`
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_partition.q
new file mode 100644
index 0000000000000000000000000000000000000000..e9416ae28222841f627aff3b962f617639c6cd99
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_partition.q
@@ -0,0 +1,24 @@
+
+set hive.support.quoted.identifiers=column;
+
+
+create table src_p(`x+1` string, `y&y` string) partitioned by (`!@#$%^&*()_q` string);
+insert overwrite table src_p partition(`!@#$%^&*()_q`='a') select * from src;
+
+show partitions src_p;
+
+explain select `x+1`, `y&y`, `!@#$%^&*()_q` 
+from src_p where `!@#$%^&*()_q` = 'a' and `x+1`='10'
+group by `x+1`, `y&y`, `!@#$%^&*()_q` having `!@#$%^&*()_q` = 'a'
+;
+
+set hive.exec.dynamic.partition.mode=nonstrict
+;
+
+create table src_p2(`x+1` string) partitioned by (`!@#$%^&*()_q` string);
+
+insert overwrite table src_p2 partition(`!@#$%^&*()_q`)
+select key, value as `!@#$%^&*()_q` from src where key < '200'
+;
+
+show partitions src_p2;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_skew.q
new file mode 100644
index 0000000000000000000000000000000000000000..5c959674117f1d189ebb120ffb096566e46f7758
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_skew.q
@@ -0,0 +1,26 @@
+
+set hive.support.quoted.identifiers=column;
+
+set hive.mapred.supports.subdirectories=true;
+set hive.internal.ddl.list.bucketing.enable=true;
+set hive.optimize.skewjoin.compiletime = true;
+
+CREATE TABLE T1(`!@#$%^&*()_q` string, `y&y` string)
+SKEWED BY (`!@#$%^&*()_q`) ON ((2)) STORED AS TEXTFILE
+;
+
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
+
+CREATE TABLE T2(`!@#$%^&*()_q` string, `y&y` string)
+SKEWED BY (`!@#$%^&*()_q`) ON ((2)) STORED AS TEXTFILE
+;
+
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T2;
+
+-- a simple join query with skew on both the tables on the join key
+-- adding a order by at the end to make the results deterministic
+
+EXPLAIN
+SELECT a.*, b.* FROM T1 a JOIN T2 b ON a. `!@#$%^&*()_q`  = b. `!@#$%^&*()_q` 
+;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_smb.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_smb.q
new file mode 100644
index 0000000000000000000000000000000000000000..38d1b99c4b8e62d147e0ee155d4751cf5cd9cc7e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_smb.q
@@ -0,0 +1,34 @@
+
+set hive.support.quoted.identifiers=column;
+
+
+set hive.enforce.bucketing = true;  
+set hive.enforce.sorting = true;  
+create table src_b(`x+1` string, `!@#$%^&*()_q` string)  
+clustered by (`!@#$%^&*()_q`) sorted by (`!@#$%^&*()_q`) into 2 buckets
+;
+
+insert overwrite table src_b
+select * from src
+;
+
+create table src_b2(`x+1` string, `!@#$%^&*()_q` string)  
+clustered by (`!@#$%^&*()_q`) sorted by (`!@#$%^&*()_q`) into 2 buckets
+;
+
+insert overwrite table src_b2
+select * from src
+;
+
+set hive.auto.convert.join=true;
+set hive.auto.convert.sortmerge.join=true;
+set hive.optimize.bucketmapjoin = true;
+set hive.optimize.bucketmapjoin.sortedmerge = true;
+
+set hive.auto.convert.sortmerge.join.to.mapjoin=false;
+set hive.auto.convert.sortmerge.join.bigtable.selection.policy = org.apache.hadoop.hive.ql.optimizer.TableSizeBasedBigTableSelectorForAutoSMJ;
+
+select a.`x+1`, a.`!@#$%^&*()_q`, b.`x+1`, b.`!@#$%^&*()_q`
+from src_b a join src_b2 b on a.`!@#$%^&*()_q` = b.`!@#$%^&*()_q`
+where a.`x+1` < '11'
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_tblproperty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_tblproperty.q
new file mode 100644
index 0000000000000000000000000000000000000000..d64e9cb9d524d29af2085fca0eed35d86916657f
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_tblproperty.q
@@ -0,0 +1,8 @@
+ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar;
+
+CREATE TABLE xyz(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' 
+STORED AS TEXTFILE
+TBLPROPERTIES('columns'='valid_colname,invalid.colname')
+;
+
+describe xyz;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q
index 3e83e6693b276a60c6f48bf170dd62c8e9768d7a..df460c89aa896eee44d2b767f2ae7ab1d7d8e0ef 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q
@@ -1,7 +1,7 @@
 set hive.map.aggr.hash.percentmemory = 0.3;
 set hive.mapred.local.mem = 256;
 
-add file ../data/scripts/dumpdata_script.py;
+add file ../../data/scripts/dumpdata_script.py;
 
 CREATE table columnTable_Bigdata (key STRING, value STRING)
 ROW FORMAT SERDE
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q
index 9cfcee5a599c659df6d34a24efbb77f5a9741474..1c311fc478203fc23f5b9918cbeb8db6e8e3f675 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q
@@ -1,3 +1,5 @@
+set hive.support.quoted.identifiers=none;
+
 EXPLAIN
 SELECT * FROM srcpart;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q
index 926601c647bc38e0ab31f156c267b558e82c1895..c4fcaaf95b6a190324e664cb26c3965ece7f1b53 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q
@@ -1,4 +1,4 @@
-dfs -put ../data/scripts/newline.py /newline.py;
+dfs -put ../../data/scripts/newline.py /newline.py;
 add file hdfs:///newline.py;
 set hive.transform.escape.input=true;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q
index 8d04d3e991c5950c8a76bacb31e27cdd5ef1cbe5..df199b0d7765b76ed9683590c303e0698afdcd89 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q
@@ -1,10 +1,12 @@
+DROP TABLE IF EXISTS repairtable;
+
 CREATE TABLE repairtable(col STRING) PARTITIONED BY (p1 STRING, p2 STRING);
 
 MSCK TABLE repairtable;
 
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/warehouse/repairtable/p1=a/p2=a;
-dfs ${system:test.dfs.mkdir} ../build/ql/test/data/warehouse/repairtable/p1=b/p2=a;
-dfs -touchz ../build/ql/test/data/warehouse/repairtable/p1=b/p2=a/datafile;
+dfs ${system:test.dfs.mkdir} ${system:test.warehouse.dir}/repairtable/p1=a/p2=a;
+dfs ${system:test.dfs.mkdir} ${system:test.warehouse.dir}/repairtable/p1=b/p2=a;
+dfs -touchz ${system:test.warehouse.dir}/repairtable/p1=b/p2=a/datafile;
 
 MSCK TABLE repairtable;
 
@@ -12,4 +14,4 @@ MSCK REPAIR TABLE repairtable;
 
 MSCK TABLE repairtable;
 
-
+DROP TABLE repairtable;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/root_dir_external_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/root_dir_external_table.q
new file mode 100644
index 0000000000000000000000000000000000000000..a0514c86ff21112366a177e6a44a3819d8c6bd62
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/root_dir_external_table.q
@@ -0,0 +1,11 @@
+dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_root_dir_external_table;
+
+insert overwrite directory "hdfs:///tmp/test_root_dir_external_table" select key from src where (key < 20) order by key;
+
+dfs -cp /tmp/test_root_dir_external_table/000000_0 /000000_0;
+dfs -rmr hdfs:///tmp/test_root_dir_external_table;
+
+create external table roottable (key string) row format delimited fields terminated by '\\t' stored as textfile location 'hdfs:///';
+select count(*) from roottable;
+
+dfs -rmr /000000_0;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q
index ecd4d13d0e23b0d5ffe51aa89a91945b9ade7165..b3c38bf5771995aca9b1cc803bafda012d251846 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q
@@ -1,5 +1,5 @@
-dfs -mkdir file:///tmp/test;
-dfs -mkdir hdfs:///tmp/test;
+dfs ${system:test.dfs.mkdir} file:///tmp/test;
+dfs ${system:test.dfs.mkdir} hdfs:///tmp/test;
 
 create external table dynPart (key string) partitioned by (value string, value2 string) row format delimited fields terminated by '\\t' stored as textfile;
 insert overwrite local directory "/tmp/test" select key from src where (key = 10) order by key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q
index 4f65016f3081c6bcea4a96c0de01a7b86cf86a88..2dfb12951f9d3884f5c9ea82b7bac13cbf485579 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q
@@ -1,7 +1,9 @@
 set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+
+-- EXCLUDE_OS_WINDOWS
 CREATE TABLE dest1(key INT, value STRING);
 
-ADD FILE src/test/scripts/testgrep;
+ADD FILE ../../ql/src/test/scripts/testgrep;
 
 FROM (
   FROM src
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1_win.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1_win.q
new file mode 100644
index 0000000000000000000000000000000000000000..0008ae51c4365276659ec3d0912417084b93646b
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1_win.q
@@ -0,0 +1,16 @@
+set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
+-- INCLUDE_OS_WINDOWS
+
+CREATE TABLE dest1(key INT, value STRING);
+
+ADD FILE src/test/scripts/testgrep_win.bat;
+
+FROM (
+  FROM src
+  SELECT TRANSFORM(src.key, src.value)
+         USING 'testgrep_win.bat' AS (tkey, tvalue)
+  CLUSTER BY tkey
+) tmap
+INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue;
+
+SELECT dest1.* FROM dest1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_dummy_source.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_dummy_source.q
new file mode 100644
index 0000000000000000000000000000000000000000..25a1a81283221f61f7a988b75de4f60d79f04d82
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_dummy_source.q
@@ -0,0 +1,33 @@
+explain
+select 'a', 100;
+select 'a', 100;
+
+--evaluation
+explain
+select 1 + 1;
+select 1 + 1;
+
+-- explode (not possible for lateral view)
+explain
+select explode(array('a', 'b'));
+select explode(array('a', 'b'));
+
+set hive.fetch.task.conversion=more;
+
+explain
+select 'a', 100;
+select 'a', 100;
+
+explain
+select 1 + 1;
+select 1 + 1;
+
+explain
+select explode(array('a', 'b'));
+select explode(array('a', 'b'));
+
+-- subquery
+explain
+select 2 + 3,x from (select 1 + 2 x) X;
+select 2 + 3,x from (select 1 + 2 x) X;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q
index 2a287bd877759701050785adcfd010b478e1ab86..accdb54744cc17819860948039112dfb50f27e01 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q
@@ -31,8 +31,8 @@ WITH SERDEPROPERTIES (
 )
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH "../data/files/apache.access.log" INTO TABLE serde_regex;
-LOAD DATA LOCAL INPATH "../data/files/apache.access.2.log" INTO TABLE serde_regex;
+LOAD DATA LOCAL INPATH "../../data/files/apache.access.log" INTO TABLE serde_regex;
+LOAD DATA LOCAL INPATH "../../data/files/apache.access.2.log" INTO TABLE serde_regex;
 
 SELECT * FROM serde_regex ORDER BY time;
 
@@ -42,7 +42,7 @@ DROP TABLE serde_regex;
 
 EXPLAIN
 CREATE TABLE serde_regex1(
-  key decimal,
+  key decimal(38,18),
   value int)
 ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe'
 WITH SERDEPROPERTIES (
@@ -51,7 +51,7 @@ WITH SERDEPROPERTIES (
 STORED AS TEXTFILE;
 
 CREATE TABLE serde_regex1(
-  key decimal,
+  key decimal(38,18),
   value int)
 ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe'
 WITH SERDEPROPERTIES (
@@ -59,7 +59,7 @@ WITH SERDEPROPERTIES (
 )
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH "../data/files/kv7.txt" INTO TABLE serde_regex1;
+LOAD DATA LOCAL INPATH "../../data/files/kv7.txt" INTO TABLE serde_regex1;
 
 SELECT key, value FROM serde_regex1 ORDER BY key, value;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q
index 7e3d1f4d8aa4194d8a7ff4b8740c07a294e417e7..d10239c31af6264f95973eafae16367e511215d3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q
@@ -24,7 +24,7 @@ set b=a;
 set c=${hiveconf:${hiveconf:b}};
 set c;
 
-set jar=${system:build.ivy.lib.dir}/default/derby-${system:derby.version}.jar;
+set jar=${system:maven.local.repository}/org/apache/derby/derby/${system:derby.version}/derby-${system:derby.version}.jar;
 
 add file ${hiveconf:jar};
 delete file ${hiveconf:jar};
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q
index 5fcdf97e2db478fcf125b1d789425c3db5d3fc73..9758c16caa5ad9e1bfea64352ec0d4b350c7b5c0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 DROP TABLE show_idx_empty;
 DROP TABLE show_idx_full;
 
@@ -24,4 +25,4 @@ SHOW INDEXES ON show_idx_empty;
 DROP INDEX idx_1 on show_idx_full;
 DROP INDEX idx_2 on show_idx_full;
 DROP TABLE show_idx_empty;
-DROP TABLE show_idx_full;
\ No newline at end of file
+DROP TABLE show_idx_full;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q
index ab588937e179b1d5b69db92dab3825298300d300..bb43c5e1387a9e7d04762bb852361b5151a6e971 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 DROP TABLE show_idx_t1;
 
 CREATE TABLE show_idx_t1(KEY STRING, VALUE STRING);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q
index 7fa7b828bd72e1abe413a818b2f7a2abac836647..1fc1d8e1f2a8beb4e59f0ad085e8eefb664e6ffb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q
@@ -1,4 +1,5 @@
 SHOW PARTITIONS srcpart;
+SHOW PARTITIONS default.srcpart;
 SHOW PARTITIONS srcpart PARTITION(hr='11');
 SHOW PARTITIONS srcpart PARTITION(ds='2008-04-08');
-SHOW PARTITIONS srcpart PARTITION(ds='2008-04-08', hr='12');
\ No newline at end of file
+SHOW PARTITIONS srcpart PARTITION(ds='2008-04-08', hr='12');
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_roles.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_roles.q
new file mode 100644
index 0000000000000000000000000000000000000000..d8ce96a37d7a875fd9544945c3c0ffd425b7435e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_roles.q
@@ -0,0 +1,4 @@
+create role role1;
+create role role2;
+
+show roles;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q
index 9184d6da897c0873666dc7702f3da19f34a7dd13..55fb7b67ffc838efe501ace1dbfb227dbb6ee60a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q
@@ -1,3 +1,4 @@
+set hive.support.quoted.identifiers=none;
 EXPLAIN 
 SHOW TABLE EXTENDED IN default LIKE `src`;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q
index ad917beeef9cef52552c43a26bba1a505011818e..47535eab638d952a7b05b69388d512b85e6eca6c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q
@@ -13,10 +13,10 @@ CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE;
 CREATE TABLE T4(key STRING, val STRING) STORED AS TEXTFILE;
 CREATE TABLE dest_j1(key INT, value STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
-LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3;
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T4;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T4;
 
 
 EXPLAIN
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_noskew.q
new file mode 100644
index 0000000000000000000000000000000000000000..b8ca592ab70a1c5bf6ad5fc9eb6dad91824c2bb2
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_noskew.q
@@ -0,0 +1,9 @@
+set hive.auto.convert.join=false;
+set hive.optimize.skewjoin=true;
+
+explain
+create table noskew as select a.* from src a join src b on a.key=b.key order by a.key limit 30;
+
+create table noskew as select a.* from src a join src b on a.key=b.key order by a.key limit 30;
+
+select * from noskew;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q
index 03eab4cd6d54eb06a0da7a527f1193c76326d96c..fc07742cd7422dbe46a58b0956574ab093ce1e5c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q
@@ -20,12 +20,12 @@ set mapred.input.dir.recursive=true;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING)
 SKEWED BY (key) ON ((3)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- a simple join query with skew on both the tables on the join key
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q
index 9cb919531f7c5915e7d9b6daa66478d28cb8233a..50cfc61962af90190e6cf066a62ff3aaee611180 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q
@@ -12,16 +12,16 @@ set mapred.input.dir.recursive=true;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key) ON ((2), (8)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING)
 SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3;
 
 -- This is to test the union->selectstar->filesink and skewjoin optimization
 -- Union of 3 map-reduce subqueries is performed for the skew join
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q
index af446bb65c08ebbdb262dd2dfe7154bbf1df75f3..504ba8be2a29e27c25c196f1f99abb917eb8c857 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q
@@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING)
 SKEWED BY (key) ON ((3)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- a simple join query with skew on both the tables on the join key
 -- adding a order by at the end to make the results deterministic
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q
index 199f3201afb777dc5405f649870d2ce90b2e5de0..f35af901704ef40305328151e9c52168bcc20e0a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q
@@ -4,7 +4,7 @@ set hive.optimize.skewjoin.compiletime = true;
 
 CREATE TABLE T1(key STRING, value STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 drop table array_valued_T1;
 create table array_valued_T1 (key string, value array<string>) SKEWED BY (key) ON ((8));
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q
index ef61fb22f17a120c8b07f85791c30906f883fe82..9e00bdcd76080a2e8bac781dde68d8e6ff8afd50 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q
@@ -5,11 +5,11 @@ set hive.optimize.skewjoin.compiletime = true;
 CREATE TABLE T1(key STRING, val STRING)	
 SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;	
        
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;	
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;	
      
 CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE;	
        
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;	
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;	
      
 -- This test is to verify the skew join compile optimization when the join is followed
 -- by a union. Both sides of a union consist of a join, which should have used
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q
index b5d9d9bc4687426ee2aa08cd760fd21c9f802f08..171995069b77bc43541e43d59cbc1fd64232e08e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q
@@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key, val) ON ((2, 12), (8, 18)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING)
 SKEWED BY (key, val) ON ((3, 13), (8, 18)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- Both the join tables are skewed by 2 keys, and one of the skewed values
 -- is common to both the tables. The join key matches the skewed key set.
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q
index 0634c4f4ff6db635194f323556ee5026adcd5b5d..5ef217c90064af901cdb977ddbf7c7130f30c7ec 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q
@@ -4,16 +4,16 @@ set hive.optimize.skewjoin.compiletime = true;
 
 CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 CREATE TABLE T3(key STRING, val STRING)
 SKEWED BY (val) ON ((12)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3;
 
 -- This test is for skewed join compile time optimization for more than 2 tables.
 -- The join key for table 3 is different from the join key used for joining
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q
index 0f031dd4fc0ae65f7191377fbaf0538371be5a62..df1a26bcc7d9dd4e9dc8e5404c16a946b213d53a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q
@@ -5,16 +5,16 @@ set hive.optimize.skewjoin.compiletime = true;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 CREATE TABLE T3(key STRING, val STRING)
 SKEWED BY (val) ON ((12)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3;
 
 -- This test is for skewed join compile time optimization for more than 2 tables.
 -- The join key for table 3 is different from the join key used for joining
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q
index d5474a455e9dd7b1cf3c4721febd690693308219..1db5472396db12ecf4b516fa7c921bdb0db4839e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q
@@ -4,7 +4,7 @@ set hive.optimize.skewjoin.compiletime = true;
 
 CREATE TABLE tmpT1(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE tmpT1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE tmpT1;
 
 -- testing skew on other data types - int
 CREATE TABLE T1(key INT, val STRING) SKEWED BY (key) ON ((2));
@@ -12,7 +12,7 @@ INSERT OVERWRITE TABLE T1 SELECT key, val FROM tmpT1;
 
 CREATE TABLE tmpT2(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE tmpT2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE tmpT2;
 
 CREATE TABLE T2(key INT, val STRING) SKEWED BY (key) ON ((3));
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q
index 46b4f6d6e600da75e90f760f6463e7b7d4d4fcfe..915de612ded5387bb636c79cb7dbc99bf1d3cf1b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q
@@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key, val) ON ((2, 12)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING)
 SKEWED BY (key) ON ((3)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- One of the tables is skewed by 2 columns, and the other table is
 -- skewed by one column. Ths join is performed on the both the columns
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q
index 0592ca8c3e4982bf79d0c27490e190d463535bfd..2ee79cc758531ea075aaed626b3b9b52ca62c59e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q
@@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key, val) ON ((2, 12)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING)
 SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- One of the tables is skewed by 2 columns, and the other table is
 -- skewed by one column. Ths join is performed on the first skewed column
@@ -31,12 +31,12 @@ DROP TABLE T2;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key, val) ON ((2, 12)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING)
 SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- One of the tables is skewed by 2 columns, and the other table is
 -- skewed by one column. Ths join is performed on the both the columns
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q
index 433fea336dfd4d6d26611eb70f49c730d0d95460..9d06cc030699a891339089def5146a34a63dbf39 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q
@@ -4,7 +4,7 @@ set hive.optimize.skewjoin.compiletime = true;
 
 CREATE TABLE tmpT1(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE tmpT1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE tmpT1;
 
 -- testing skew on other data types - int
 CREATE TABLE T1(key INT, val STRING) SKEWED BY (key) ON ((2));
@@ -16,7 +16,7 @@ INSERT OVERWRITE TABLE T1 SELECT key, val FROM tmpT1;
 CREATE TABLE T2(key STRING, val STRING)
 SKEWED BY (key) ON ((3)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- Once HIVE-3445 is fixed, the compile time skew join optimization would be
 -- applicable here. Till the above jira is fixed, it would be performed as a
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q
index 0b11ebe4cb696adbb44a805051ad73e52ee8ff5a..075645f89d4521c26083e02eb9b017ce0bd864fb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q
@@ -6,11 +6,11 @@ CREATE TABLE T1(key STRING, val STRING)
 CLUSTERED BY (key) INTO 4 BUCKETS
 SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- add a test where the skewed key is also the bucketized key
 -- it should not matter, and the compile time skewed join
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q
index 34fcdbfac4cb76fdd50ac2d3846a798c42cd6ff2..f7acaad18e1ea5402e11a55b21c0e3a9d87c2370 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q
@@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key) ON ((2), (7)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING)
 SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- a simple query with skew on both the tables on the join key
 -- multiple skew values are present for the skewed keys
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q
index f217052881e24fc695f26b3fa35d17297a524500..9b908ce21b1d2e2e5b5dadf2a29f09714251a6f2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q
@@ -6,11 +6,11 @@ CREATE TABLE T1(key STRING, val STRING)
 CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS
 SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- add a test where the skewed key is also the bucketized/sorted key
 -- it should not matter, and the compile time skewed join
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q
index f6002ad49802a4087f271288880b6e743a60a372..22ea4f06218ac5a58d5a3d228beea03bbe61c595 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q
@@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key) ON ((2), (8)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING)
 SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- a simple query with skew on both the tables. One of the skewed
 -- value is common to both the tables. The skewed value should not be
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q
index ca83c446085fa2b5bd622aeea734feecd18266f2..8496b1aa79c0a6852db1ed72768fd5e9837b7f4c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q
@@ -5,11 +5,11 @@ set hive.optimize.skewjoin.compiletime = true;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- only of the tables of the join (the left table of the join) is skewed
 -- the skewed filter would still be applied to both the tables
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q
index 3d7884c5e3dcb0a2f7548bc79a6e0efcf0f63ac1..152de5bde72c5ab68a3f80f7557f04a7dca99d2b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q
@@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key, val) ON ((2, 12)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING)
 SKEWED BY (key) ON ((3)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- One of the tables is skewed by 2 columns, and the other table is
 -- skewed by one column. Ths join is performed on the first skewed column
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q
index 36cf8ceeaebb49a943fdf1a40c7aba65e568a14a..2e261bde66bbf3727568d7167e11df789c538618 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q
@@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key, val) ON ((2, 12), (8, 18)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING)
 SKEWED BY (key, val) ON ((3, 13), (8, 18)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- Both the join tables are skewed by 2 keys, and one of the skewed values
 -- is common to both the tables. The join key is a subset of the skewed key set:
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q
index cf84f67b6a0fc92eb5b7ab2b12ef1d958f45fd58..e4d9605f6f7afd3513883588a44d0eeeffb004a1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q
@@ -5,16 +5,16 @@ set hive.optimize.skewjoin.compiletime = true;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key) ON ((2), (8)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING)
 SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3;
 
 -- This test is for validating skewed join compile time optimization for more than
 -- 2 tables. The join key is the same, and so a 3-way join would be performed.
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q
index d0ac845f86581ab5d70c5c92bb6887fbd3168a43..85746d9611dab284912c43a91178f2bd7d49d26d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q
@@ -4,16 +4,16 @@ set hive.optimize.skewjoin.compiletime = true;
 
 CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING)
 SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3;
 
 -- This test is for validating skewed join compile time optimization for more than
 -- 2 tables. The join key is the same, and so a 3-way join would be performed.
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q
index 04834033a11e643b9e6e46a410884284206eb419..889ab6c3f55347a5ef87b5bf7270a63683eee579 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q
@@ -5,11 +5,11 @@ set hive.optimize.skewjoin.compiletime = true;
 CREATE TABLE T1(key STRING, val STRING)
 SKEWED BY (key) ON ((2)) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
 
 -- no skew join compile time optimization would be performed if one of the
 -- join sources is a sub-query consisting of a union all
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q
index 359513e424dba1e50280985aa92526012d48c0b7..9dee4110f59916795311bbbf8c694a2f4d70bf3d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q
@@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k
 create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; 
 create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE;
 
-load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1;
-load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2;
-load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3;
+load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1;
+load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2;
+load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3;
 
 set hive.optimize.bucketmapjoin = true;
 set hive.optimize.bucketmapjoin.sortedmerge = true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q
index a79ebf62d0693b99ba894eb91564162172776869..1fbe2090eaf2a71124d1b41ba3e65908b4885f20 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q
@@ -6,11 +6,11 @@ alter table tmp_smb_bucket_10 add partition (ds = '2');
 
 -- add dummy files to make sure that the number of files in each partition is same as number of buckets
  
-load data local inpath '../data/files/smbbucket_1.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='1');
-load data local inpath '../data/files/smbbucket_2.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='1');
+load data local inpath '../../data/files/smbbucket_1.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='1');
+load data local inpath '../../data/files/smbbucket_2.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='1');
 
-load data local inpath '../data/files/smbbucket_1.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='2');
-load data local inpath '../data/files/smbbucket_2.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='2');
+load data local inpath '../../data/files/smbbucket_1.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='2');
+load data local inpath '../../data/files/smbbucket_2.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='2');
 
 set hive.optimize.bucketmapjoin = true;
 set hive.optimize.bucketmapjoin.sortedmerge = true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q
index 9d86314879d684c63f125478b4bfca45809d7e48..e2b24333ad4162cf99156c4cc680c9fc03ce382f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q
@@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k
 create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; 
 create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE;
 
-load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1;
-load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2;
-load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3;
+load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1;
+load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2;
+load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3;
  
 set hive.optimize.bucketmapjoin = true;
 set hive.optimize.bucketmapjoin.sortedmerge = true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q
index 8b534e85aee1d7df1349992ce8a5088b7c6d4599..e43174bc0768b42dc23e006ac6d13b1c0ac88ec3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q
@@ -10,9 +10,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k
 create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; 
 create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE;
 
-load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1;
-load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2;
-load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3;
+load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1;
+load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2;
+load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3;
 
 explain 
 select * from (select a.key from smb_bucket_1 a join smb_bucket_2 b on (a.key = b.key) where a.key = 5) t1 left outer join (select c.key from smb_bucket_2 c join smb_bucket_3 d on (c.key = d.key) where c.key=5) t2 on (t1.key=t2.key) where t2.key=5;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q
index 73b21fae250e0e055a679f04476155e22c091a4e..b379706cc8ac591865eecb2b0dda666434ec18d7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q
@@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k
 create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; 
 create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE;
 
-load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1;
-load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2;
-load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3;
+load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1;
+load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2;
+load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3;
 
 set hive.optimize.bucketmapjoin = true;
 set hive.optimize.bucketmapjoin.sortedmerge = true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q
index 83143b170ed5cb97280fcf327ef6b5aed9668145..2b3f67ea4eea195090ab7c4aa0e5d564ca90c0e1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q
@@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k
 create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; 
 create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE;
 
-load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1;
-load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2;
-load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3;
+load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1;
+load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2;
+load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3;
 
 set hive.optimize.bucketmapjoin = true;
 set hive.optimize.bucketmapjoin.sortedmerge = true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q
index 61ec084f64ffa7b6fed847f7f9bd284519f6e630..406604e621ad0157c358d5c3be7495ad01f41fd5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q
@@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k
 create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; 
 create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE;
 
-load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1;
-load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2;
-load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3;
+load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1;
+load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2;
+load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3;
 
 set hive.optimize.bucketmapjoin = true;
 set hive.optimize.bucketmapjoin.sortedmerge = true;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q
index 1488b1f949527d8545d5bc389f821f53abfa07b5..ca1c7491b72981984e3f89b04caee76633b2be6b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q
@@ -15,8 +15,8 @@ create table smb_join_results(k1 int, v1 string, k2 int, v2 string);
 create table smb_join_results_empty_bigtable(k1 int, v1 string, k2 int, v2 string);
 create table normal_join_results(k1 int, v1 string, k2 int, v2 string);
 
-load data local inpath '../data/files/empty1.txt' into table smb_bucket4_1;
-load data local inpath '../data/files/empty2.txt' into table smb_bucket4_1;
+load data local inpath '../../data/files/empty1.txt' into table smb_bucket4_1;
+load data local inpath '../../data/files/empty2.txt' into table smb_bucket4_1;
 
 insert overwrite table smb_bucket4_2
 select * from src;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q
index 6f282ed441bfa1144024069262f8ba8379703ff6..f296057d43e3820b01bb8e68a0c91559b623b4f0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q
@@ -5,7 +5,7 @@ set hive.exec.reducers.max = 1;
 
 
 create table smb_bucket_input (key int, value string) stored as rcfile;
-load data local inpath '../data/files/smb_bucket_input.rc' into table smb_bucket_input;
+load data local inpath '../../data/files/smb_bucket_input.rc' into table smb_bucket_input;
 
 
 CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q
index 6fe3d211a00b0b9c751d2aa6c9ece74a2c2c1cab..76ca152ef55f4064813f9c55f447846a62f0504f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q
@@ -1 +1 @@
-source ../data/files/source.txt;
+source ../../data/files/source.txt;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split.q
new file mode 100644
index 0000000000000000000000000000000000000000..f5d7ff8fdd9bf449a079ce56cd8a7ee3762a7c64
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split.q
@@ -0,0 +1,8 @@
+DROP TABLE tmp_jo_tab_test;
+CREATE table tmp_jo_tab_test (message_line STRING)
+STORED AS TEXTFILE;
+
+LOAD DATA LOCAL INPATH '../../data/files/input.txt'
+OVERWRITE INTO TABLE tmp_jo_tab_test;
+
+select size(split(message_line, '\t')) from tmp_jo_tab_test;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q
index 0b783de153b2907aaa9dbc2805cb4bb2d9912e01..359d27b31523ac0e0dab7a1a9e711c623bc9abdd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q
@@ -26,5 +26,5 @@ DESCRIBE FORMATTED tmptable;
 -- Load a file into a existing table
 -- Some stats (numFiles, totalSize) should be updated correctly
 -- Some other stats (numRows, rawDataSize) should be cleared
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE tmptable;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE tmptable;
 DESCRIBE FORMATTED tmptable;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q
index 6618c913ea7000032abe63d2ef042ace86d7dce0..d037c003b75822a0c06d27b85d2ef52cb423e1d0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q
@@ -2,25 +2,25 @@ set datanucleus.cache.collections=false;
 set hive.stats.autogather=true;
 
 CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin;
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin;
 
 CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
 explain
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
 
 desc formatted srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
 desc formatted srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
 desc formatted srcbucket_mapjoin_part partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08');
 desc formatted srcbucket_mapjoin_part partition(ds='2008-04-08');
 
 CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
-load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08');
 
 create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint);
 create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q
index 425de64c26e838e3bf456dc446d98dfd7f0e244d..e773cd749403f8cb600bdbca573411e1565f71de 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q
@@ -13,7 +13,7 @@ insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select
 -- Some other stats (numRows, rawDataSize) should be cleared
 desc formatted stats_part partition (ds='2010-04-08', hr='13');
 
-load data local inpath '../data/files/srcbucket20.txt' INTO TABLE stats_part partition (ds='2010-04-08', hr='13');
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE stats_part partition (ds='2010-04-08', hr='13');
 
 desc formatted stats_part partition (ds='2010-04-08', hr='13');
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q
index da4af9655d16a76b8f33dbc64e9e6733e9893218..51514bd7738ff37eb8d43a12bd13acc8ae6039ac 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q
@@ -1,7 +1,7 @@
 set datanucleus.cache.collections=false;
 set hive.stats.autogather=true;
 set hive.stats.reliable=true;
-set hive.stats.dbclass=dummy;
+set hive.stats.dbclass=custom;
 set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher;
 set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.KeyVerifyingStatsAggregator;
 
@@ -56,7 +56,7 @@ insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select
 desc formatted stats_part partition (ds='2010-04-08', hr = '13');
 
 
-set hive.stats.dbclass=dummy;
+set hive.stats.dbclass=custom;
 set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher;
 set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.KeyVerifyingStatsAggregator;
 set hive.stats.key.prefix.max.length=0;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q
index 5962348d9c31746449badb43b83d5e1f3e9b4528..fd7e0eaca8c39623f94c6f6398a51b8d3227bc91 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q
@@ -5,9 +5,9 @@ drop table hive_test_dst;
 
 create table hive_test_src ( col1 string ) stored as textfile ;
 explain extended
-load data local inpath '../data/files/test.dat' overwrite into table hive_test_src ;
+load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ;
 
-load data local inpath '../data/files/test.dat' overwrite into table hive_test_src ;
+load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ;
 
 desc formatted hive_test_src;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q
index 62580042d4deda9a72a1094d40bc40bb78341fe6..80a67f405cb0bd71c857c7bba6935d2d2cedea57 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q
@@ -25,8 +25,8 @@ insert overwrite table nzhang_part2 partition(ds='2008-12-31', hr) select key, v
 show partitions nzhang_part1;
 show partitions nzhang_part2;
 
-select * from nzhang_part1 where ds is not null and hr is not null;
-select * from nzhang_part2 where ds is not null and hr is not null;
+select * from nzhang_part1 where ds is not null and hr is not null order by ds, hr, key;
+select * from nzhang_part2 where ds is not null and hr is not null order by ds, hr, key;
 
 describe formatted nzhang_part1 partition(ds='2008-04-08',hr=11);
 describe formatted nzhang_part1 partition(ds='2008-04-08',hr=12);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q
index 4e7d3dc547a30e7b9a389e68e0364e523edca197..5e6b0aaa1253be070f14a870e93acc43739d4d7e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q
@@ -1,12 +1,12 @@
 -- In this test, there is a dummy stats aggregator which throws an error when various
--- methods are called (as indicated by the parameter hive.test.dummystats.agregator)
+-- methods are called (as indicated by the parameter hive.test.dummystats.aggregator)
 -- Since stats need not be reliable (by setting hive.stats.reliable to false), the 
 -- insert statements succeed. The insert statement succeeds even if the stats aggregator
 -- is set to null, since stats need not be reliable.
 
 create table tmptable(key string, value string);
 
-set hive.stats.dbclass=dummy;
+set hive.stats.dbclass=custom;
 set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher;
 set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator;
 set hive.stats.reliable=false;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter.q
new file mode 100644
index 0000000000000000000000000000000000000000..3c1f132a68f2f5d546b5c6f681427330253e15a5
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter.q
@@ -0,0 +1,16 @@
+set hive.stats.dbclass=counter;
+set hive.stats.autogather=false;
+
+-- by analyze
+create table dummy1 as select * from src;
+
+analyze table dummy1 compute statistics;
+desc formatted dummy1;
+
+set hive.stats.dbclass=counter;
+set hive.stats.autogather=true;
+
+-- by autogather
+create table dummy2 as select * from src;
+
+desc formatted dummy2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter_partitioned.q
new file mode 100644
index 0000000000000000000000000000000000000000..e1274c0cb5197fe48fb53accaf1eda0a03587f10
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter_partitioned.q
@@ -0,0 +1,45 @@
+set hive.stats.dbclass=counter;
+set hive.stats.autogather=true;
+set hive.exec.dynamic.partition.mode=nonstrict;
+
+-- partitioned table analyze 
+
+create table dummy (key string, value string) partitioned by (ds string, hr string);
+
+load data local inpath '../../data/files/kv1.txt' into table dummy partition (ds='2008',hr='12');
+load data local inpath '../../data/files/kv1.txt' into table dummy partition (ds='2008',hr='11');
+
+analyze table dummy partition (ds,hr) compute statistics;
+describe formatted dummy partition (ds='2008', hr='11');
+describe formatted dummy partition (ds='2008', hr='12');
+
+drop table dummy;
+
+-- static partitioned table on insert
+
+create table dummy (key string, value string) partitioned by (ds string, hr string);
+
+insert overwrite table dummy partition (ds='10',hr='11') select * from src;
+insert overwrite table dummy partition (ds='10',hr='12') select * from src;
+
+describe formatted dummy partition (ds='10', hr='11');
+describe formatted dummy partition (ds='10', hr='12');
+
+drop table dummy;
+
+-- dynamic partitioned table on insert
+
+create table dummy (key int) partitioned by (hr int);
+                                                                                                      
+CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|';
+LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE INTO TABLE tbl;                           
+                                                                                                      
+insert overwrite table dummy partition (hr) select * from tbl;
+
+describe formatted dummy partition (hr=1997);
+describe formatted dummy partition (hr=1994);
+describe formatted dummy partition (hr=1998);
+describe formatted dummy partition (hr=1996);
+
+drop table tbl;
+drop table dummy; 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_invalidation.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_invalidation.q
new file mode 100644
index 0000000000000000000000000000000000000000..a7fce6e3e503f9784413f4b675bcbb02619b601f
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_invalidation.q
@@ -0,0 +1,15 @@
+set hive.stats.autogather=true;
+
+CREATE TABLE stats_invalid (key string, value string);
+
+insert overwrite table stats_invalid
+select * from src;
+
+analyze table stats_invalid compute statistics for columns key,value;
+
+desc formatted stats_invalid;
+alter table stats_invalid add  columns (new_col string);
+
+desc formatted stats_invalid;
+drop table stats_invalid;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_list_bucket.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_list_bucket.q
new file mode 100644
index 0000000000000000000000000000000000000000..59826437415483d0f65f398342be965dd4d4938c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_list_bucket.q
@@ -0,0 +1,45 @@
+
+set hive.mapred.supports.subdirectories=true;
+
+-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23)
+
+drop table stats_list_bucket;
+drop table stats_list_bucket_1;
+
+create table stats_list_bucket (
+  c1 string,
+  c2 string
+) partitioned by (ds string, hr string)
+skewed by (c1, c2) on  (('466','val_466'),('287','val_287'),('82','val_82'))
+stored as directories
+stored as rcfile;
+
+set hive.stats.key.prefix.max.length=1;
+
+-- Make sure we use hashed IDs during stats publishing.
+-- Try partitioned table with list bucketing.
+-- The stats should show 500 rows loaded, as many rows as the src table has.
+
+insert overwrite table stats_list_bucket partition (ds = '2008-04-08',  hr = '11')
+  select key, value from src;
+
+desc formatted stats_list_bucket partition (ds = '2008-04-08',  hr = '11');
+
+-- Also try non-partitioned table with list bucketing.
+-- Stats should show the same number of rows.
+
+create table stats_list_bucket_1 (
+  c1 string,
+  c2 string
+)
+skewed by (c1, c2) on  (('466','val_466'),('287','val_287'),('82','val_82'))
+stored as directories
+stored as rcfile;
+
+insert overwrite table stats_list_bucket_1
+  select key, value from src;
+
+desc formatted stats_list_bucket_1;
+
+drop table stats_list_bucket;
+drop table stats_list_bucket_1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q
index c934fb205125867cb60cbcf1a5fd8fc01ad596ad..b106b30476c00a9fa77f628989d74502fc0544d7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q
@@ -1,12 +1,12 @@
 
 -- test analyze table compute statistiscs [noscan] on external table 
 -- 1 test table
-CREATE EXTERNAL TABLE anaylyze_external (a INT) LOCATION '${system:test.src.data.dir}/files/ext_test';
+CREATE EXTERNAL TABLE anaylyze_external (a INT) LOCATION '${system:hive.root}/data/files/ext_test';
 SELECT * FROM anaylyze_external;
-analyze table anaylyze_external compute statistics;
-describe formatted anaylyze_external;
 analyze table anaylyze_external compute statistics noscan;
 describe formatted anaylyze_external;
+analyze table anaylyze_external compute statistics;
+describe formatted anaylyze_external;
 drop table anaylyze_external;
 
 -- 2 test partition
@@ -21,10 +21,10 @@ CREATE EXTERNAL TABLE anaylyze_external (key string, val string) partitioned by
 ALTER TABLE anaylyze_external ADD PARTITION (insertdate='2008-01-01') location 'pfile://${system:test.tmp.dir}/texternal/2008-01-01';
 select count(*) from anaylyze_external where insertdate='2008-01-01';
 -- analyze
-analyze table anaylyze_external PARTITION (insertdate='2008-01-01') compute statistics;
-describe formatted anaylyze_external PARTITION (insertdate='2008-01-01');
 analyze table anaylyze_external PARTITION (insertdate='2008-01-01') compute statistics noscan;
 describe formatted anaylyze_external PARTITION (insertdate='2008-01-01');
+analyze table anaylyze_external PARTITION (insertdate='2008-01-01') compute statistics;
+describe formatted anaylyze_external PARTITION (insertdate='2008-01-01');
 dfs -rmr ${system:test.tmp.dir}/texternal;
 drop table anaylyze_external;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_only_null.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_only_null.q
new file mode 100644
index 0000000000000000000000000000000000000000..b47bc48958c8d2853abe8e1ca1d303f524768560
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_only_null.q
@@ -0,0 +1,41 @@
+set hive.stats.dbclass=fs;
+set hive.compute.query.using.stats=true;
+set hive.stats.autogather=true;
+CREATE TABLE temps_null(a double, b int, c STRING, d smallint) STORED AS TEXTFILE; 
+
+CREATE TABLE stats_null(a double, b int, c STRING, d smallint) STORED AS TEXTFILE; 
+
+CREATE TABLE stats_null_part(a double, b int, c STRING, d smallint) partitioned by (dt string) STORED AS TEXTFILE; 
+
+LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE temps_null;
+
+insert overwrite table stats_null select * from temps_null;
+insert into table stats_null_part partition(dt='2010') select * from temps_null where d <=5;
+
+insert into table stats_null_part partition(dt='2011') select * from temps_null where d > 5;
+explain 
+select count(*), count(a), count(b), count(c), count(d) from stats_null;
+explain 
+select count(*), count(a), count(b), count(c), count(d) from stats_null_part;
+
+
+analyze table stats_null compute statistics for columns a,b,c,d;
+analyze table stats_null_part partition(dt='2010') compute statistics for columns a,b,c,d;
+analyze table stats_null_part partition(dt='2011') compute statistics for columns a,b,c,d;
+
+describe formatted stats_null_part partition (dt='2010');
+describe formatted stats_null_part partition (dt='2011');
+
+explain 
+select count(*), count(a), count(b), count(c), count(d) from stats_null;
+explain 
+select count(*), count(a), count(b), count(c), count(d) from stats_null_part;
+
+
+select count(*), count(a), count(b), count(c), count(d) from stats_null;
+select count(*), count(a), count(b), count(c), count(d) from stats_null_part;
+drop table stats_null;
+drop table stats_null_part;
+drop table temps_null;
+set hive.compute.query.using.stats=false;
+set hive.stats.dbclass=jdbc:derby;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q
index 6d383f213d1be25992f67384b0dcf3b734bedd14..513b8e75a0c5e91460106dd64b66a11193f65a78 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q
@@ -6,7 +6,7 @@
 
 create table tmptable(key string, value string);
 
-set hive.stats.dbclass=dummy;
+set hive.stats.dbclass=custom;
 set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher;
 set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator;
 set hive.stats.reliable=false;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/statsfs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/statsfs.q
new file mode 100644
index 0000000000000000000000000000000000000000..82a2295ac27b4688cb8c15aa3ec069e659a8c49d
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/statsfs.q
@@ -0,0 +1,63 @@
+set hive.stats.dbclass=fs;
+
+-- stats computation on partitioned table with analyze command
+
+create table t1 (key string, value string) partitioned by (ds string);
+load data local inpath '../../data/files/kv1.txt' into table t1 partition (ds = '2010');
+load data local inpath '../../data/files/kv1.txt' into table t1 partition (ds = '2011');
+
+analyze table t1 partition (ds) compute statistics;
+
+describe formatted t1 partition (ds='2010');
+describe formatted t1 partition (ds='2011');
+
+drop table t1;
+
+-- stats computation on partitioned table with autogather on insert query
+
+create table t1 (key string, value string) partitioned by (ds string);
+
+insert into table t1 partition (ds='2010') select * from src;
+insert into table t1 partition (ds='2011') select * from src;
+
+describe formatted t1 partition (ds='2010');
+describe formatted t1 partition (ds='2011');
+
+drop table t1;
+
+-- analyze stmt on unpartitioned table
+
+create table t1 (key string, value string); 
+load data local inpath '../../data/files/kv1.txt' into table t1; 
+
+analyze table t1 compute statistics;
+
+describe formatted t1 ;
+
+drop table t1;
+
+-- stats computation on unpartitioned table with autogather on insert query
+
+create table t1 (key string, value string); 
+
+insert into table t1  select * from src;
+
+describe formatted t1 ;
+
+drop table t1;
+
+-- stats computation on partitioned table with autogather on insert query with dynamic partitioning
+
+
+create table t1 (key string, value string) partitioned by (ds string, hr string);
+
+set hive.exec.dynamic.partition.mode=nonstrict;
+insert into table t1 partition (ds,hr) select * from srcpart;
+
+describe formatted t1 partition (ds='2008-04-08',hr='11');
+describe formatted t1 partition (ds='2008-04-09',hr='12');
+
+drop table t1;
+set hive.exec.dynamic.partition.mode=strict;
+
+set hive.stats.dbclass=jdbc:derby;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q
index c3b206bba63250b187d26607cba3470d6a604547..ae83407f8433323894facc59cb05859185e832ee 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 desc function str_to_map;
 desc function extended str_to_map;
 
@@ -19,7 +21,7 @@ limit 3;
 
 
 drop table tbl_s2m;
-create table tbl_s2m as select 'ABC=CC_333=444' as t from src limit 3;
+create table tbl_s2m as select 'ABC=CC_333=444' as t from src tablesample (3 rows);
 
 select str_to_map(t,'_','=')['333'] from tbl_s2m;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q
index 3fb1558a90f0b2e1116b42776043aaf96e924d9a..14fa321c11c2f2ce0bc825438140685e615fdaff 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q
@@ -2,12 +2,12 @@ EXPLAIN
 FROM (
   FROM src select src.* WHERE src.key < 100
 ) unioninput
-INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.*;
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*;
 
 FROM (
   FROM src select src.* WHERE src.key < 100
 ) unioninput
-INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.*;
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*;
 
-dfs -cat ../build/ql/test/data/warehouse/union.out/*;
+dfs -cat ${system:test.warehouse.dir}/union.out/*;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq_where_serialization.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq_where_serialization.q
new file mode 100644
index 0000000000000000000000000000000000000000..1d539825424f064ce8255d2a3bac8271489e61fd
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq_where_serialization.q
@@ -0,0 +1,5 @@
+set hive.auto.convert.join=true;
+set hive.auto.convert.join.noconditionaltask.size=10000000;
+explain select src.key from src where src.key in ( select distinct key from src);
+
+set hive.auto.convert.join=false;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_alias.q
new file mode 100644
index 0000000000000000000000000000000000000000..ffc33dc2cf8523088203ea8f63f64833e58fe288
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_alias.q
@@ -0,0 +1,16 @@
+EXPLAIN
+FROM (
+  FROM src select src.* WHERE src.key < 100
+) as unioninput
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*;
+
+EXPLAIN
+SELECT * FROM
+( SELECT * FROM 
+   ( SELECT * FROM src as s ) as src1 
+) as src2;
+
+SELECT * FROM
+( SELECT * FROM 
+   ( SELECT * FROM src as s ) as src1 
+) as src2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists.q
new file mode 100644
index 0000000000000000000000000000000000000000..f812e360700232d2eff1d0fb66b6013bca0ac621
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists.q
@@ -0,0 +1,45 @@
+
+
+-- no agg, corr
+explain
+select * 
+from src b 
+where exists 
+  (select a.key 
+  from src a 
+  where b.value = a.value  and a.key = b.key and a.value > 'val_9'
+  )
+;
+
+select * 
+from src b 
+where exists 
+  (select a.key 
+  from src a 
+  where b.value = a.value  and a.key = b.key and a.value > 'val_9'
+  )
+;
+
+-- view test
+create view cv1 as 
+select * 
+from src b 
+where exists
+  (select a.key 
+  from src a 
+  where b.value = a.value  and a.key = b.key and a.value > 'val_9')
+;
+
+select * from cv1
+;
+
+-- sq in from
+select * 
+from (select * 
+      from src b 
+      where exists 
+          (select a.key 
+          from src a 
+          where b.value = a.value  and a.key = b.key and a.value > 'val_9')
+     ) a
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists_having.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists_having.q
new file mode 100644
index 0000000000000000000000000000000000000000..690aa10527a89ccae626ef016127f2990cdaf1eb
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists_having.q
@@ -0,0 +1,60 @@
+
+
+-- no agg, corr
+explain
+select b.key, count(*) 
+from src b 
+group by b.key
+having exists 
+  (select a.key 
+  from src a 
+  where a.key = b.key and a.value > 'val_9'
+  )
+;
+
+select b.key, count(*) 
+from src b 
+group by b.key
+having exists 
+  (select a.key 
+  from src a 
+  where a.key = b.key and a.value > 'val_9'
+  )
+;
+
+-- view test
+create view cv1 as 
+select b.key, count(*) as c
+from src b
+group by b.key
+having exists
+  (select a.key
+  from src a
+  where a.key = b.key and a.value > 'val_9'
+  )
+;
+
+select * from cv1;
+
+-- sq in from
+select *
+from (select b.key, count(*) 
+  from src b 
+  group by b.key
+  having exists 
+    (select a.key 
+    from src a 
+    where a.key = b.key and a.value > 'val_9'
+    )
+) a
+;
+
+-- join on agg
+select b.key, min(b.value)
+from src b
+group by b.key
+having exists ( select a.key
+                from src a
+                where a.value > 'val_9' and a.value = min(b.value)
+                )
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in.q
new file mode 100644
index 0000000000000000000000000000000000000000..69f40f9b8ca9a1410760d5219bae21be0a2cdcdf
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in.q
@@ -0,0 +1,163 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+DROP TABLE lineitem;
+CREATE TABLE lineitem (L_ORDERKEY      INT,
+                                L_PARTKEY       INT,
+                                L_SUPPKEY       INT,
+                                L_LINENUMBER    INT,
+                                L_QUANTITY      DOUBLE,
+                                L_EXTENDEDPRICE DOUBLE,
+                                L_DISCOUNT      DOUBLE,
+                                L_TAX           DOUBLE,
+                                L_RETURNFLAG    STRING,
+                                L_LINESTATUS    STRING,
+                                l_shipdate      STRING,
+                                L_COMMITDATE    STRING,
+                                L_RECEIPTDATE   STRING,
+                                L_SHIPINSTRUCT  STRING,
+                                L_SHIPMODE      STRING,
+                                L_COMMENT       STRING)
+ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|';
+
+LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem;
+
+-- non agg, non corr
+explain
+ select * 
+from src 
+where src.key in (select key from src s1 where s1.key > '9')
+;
+
+select * 
+from src 
+where src.key in (select key from src s1 where s1.key > '9')
+order by key
+;
+
+-- non agg, corr
+explain 
+select * 
+from src b 
+where b.key in
+        (select a.key 
+         from src a 
+         where b.value = a.value and a.key > '9'
+        )
+;
+
+select * 
+from src b 
+where b.key in
+        (select a.key 
+         from src a 
+         where b.value = a.value and a.key > '9'
+        )
+order by b.key
+;
+
+-- agg, non corr
+explain
+select p_name, p_size 
+from 
+part where part.p_size in 
+	(select avg(p_size) 
+	 from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a 
+	 where r <= 2
+	)
+;
+select p_name, p_size 
+from 
+part where part.p_size in 
+	(select avg(p_size) 
+	 from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a 
+	 where r <= 2
+	)
+order by p_name
+;
+
+-- agg, corr
+explain
+select p_mfgr, p_name, p_size 
+from part b where b.p_size in 
+	(select min(p_size) 
+	 from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a 
+	 where r <= 2 and b.p_mfgr = a.p_mfgr
+	)
+;
+
+select p_mfgr, p_name, p_size 
+from part b where b.p_size in 
+	(select min(p_size) 
+	 from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a 
+	 where r <= 2 and b.p_mfgr = a.p_mfgr
+	)
+order by p_mfgr, p_name, p_size 
+;
+
+-- distinct, corr
+explain 
+select * 
+from src b 
+where b.key in
+        (select distinct a.key 
+         from src a 
+         where b.value = a.value and a.key > '9'
+        )
+;
+
+select * 
+from src b 
+where b.key in
+        (select distinct a.key 
+         from src a 
+         where b.value = a.value and a.key > '9'
+        )
+order by b.key
+;
+
+-- non agg, non corr, windowing
+select p_mfgr, p_name, p_size 
+from part 
+where part.p_size in 
+  (select first_value(p_size) over(partition by p_mfgr order by p_size) from part)
+order by p_mfgr, p_name, p_size 
+;
+
+-- non agg, non corr, with join in Parent Query
+explain
+select p.p_partkey, li.l_suppkey 
+from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey 
+where li.l_linenumber = 1 and
+ li.l_orderkey in (select l_orderkey from lineitem where l_shipmode = 'AIR')
+;
+
+select p.p_partkey, li.l_suppkey 
+from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey 
+where li.l_linenumber = 1 and
+ li.l_orderkey in (select l_orderkey from lineitem where l_shipmode = 'AIR')
+order by p.p_partkey, li.l_suppkey 
+;
+
+-- non agg, corr, with join in Parent Query
+select p.p_partkey, li.l_suppkey 
+from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey 
+where li.l_linenumber = 1 and
+ li.l_orderkey in (select l_orderkey from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber)
+order by p.p_partkey, li.l_suppkey 
+;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in_having.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in_having.q
new file mode 100644
index 0000000000000000000000000000000000000000..84045568f450197cd78b5179b7ce53a54da76a5b
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in_having.q
@@ -0,0 +1,104 @@
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+-- non agg, non corr
+explain
+ select key, count(*) 
+from src 
+group by key
+having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key )
+;
+
+
+select s1.key, count(*) from src s1 where s1.key > '9' group by s1.key order by s1.key;
+
+select key, count(*) 
+from src 
+group by key
+having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key )
+order by key
+;
+
+-- non agg, corr
+explain
+ select key, value, count(*) 
+from src b
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9'  and s1.value = b.value group by s1.key )
+;
+
+-- agg, non corr
+explain
+select p_mfgr, avg(p_size)
+from part b
+group by b.p_mfgr
+having b.p_mfgr in 
+   (select p_mfgr 
+    from part
+    group by p_mfgr
+    having max(p_size) - min(p_size) < 20
+   )
+;
+
+-- join on agg
+select b.key, min(b.value)
+from src b
+group by b.key
+having b.key in ( select a.key
+                from src a
+                where a.value > 'val_9' and a.value = min(b.value)
+                )
+order by b.key
+;
+
+-- where and having
+-- Plan is:
+-- Stage 1: b semijoin sq1:src (subquery in where)
+-- Stage 2: group by Stage 1 o/p
+-- Stage 5: group by on sq2:src (subquery in having)
+-- Stage 6: Stage 2 o/p semijoin Stage 5
+explain
+select key, value, count(*) 
+from src b
+where b.key in (select key from src where src.key > '8')
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key )
+;
+
+set hive.auto.convert.join=true;
+-- Plan is:
+-- Stage  5: group by on sq2:src (subquery in having)
+-- Stage 10: hashtable for sq1:src (subquery in where)
+-- Stage  2: b map-side semijoin Stage 10 o/p
+-- Stage  3: Stage 2 semijoin Stage 5
+-- Stage  9: construct hastable for Stage 5 o/p
+-- Stage  6: Stage 2 map-side semijoin Stage 9
+explain
+select key, value, count(*) 
+from src b
+where b.key in (select key from src where src.key > '8')
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key )
+;
+
+-- non agg, non corr, windowing
+explain
+select p_mfgr, p_name, avg(p_size) 
+from part 
+group by p_mfgr, p_name
+having p_name in 
+  (select first_value(p_name) over(partition by p_mfgr order by p_size) from part)
+;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_multiinsert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_multiinsert.q
new file mode 100644
index 0000000000000000000000000000000000000000..ed36d9ef6e96105ec6e36325267192cbe3884d21
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_multiinsert.q
@@ -0,0 +1,82 @@
+set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook;
+
+CREATE TABLE src_4(
+  key STRING, 
+  value STRING
+)
+;
+
+CREATE TABLE src_5( 
+  key STRING, 
+  value STRING
+)
+;
+
+explain
+from src b 
+INSERT OVERWRITE TABLE src_4 
+  select * 
+  where b.key in 
+   (select a.key 
+    from src a 
+    where b.value = a.value and a.key > '9'
+   ) 
+INSERT OVERWRITE TABLE src_5 
+  select *  
+  where b.key not in  ( select key from src s1 where s1.key > '2') 
+  order by key 
+;
+
+from src b 
+INSERT OVERWRITE TABLE src_4 
+  select * 
+  where b.key in 
+   (select a.key 
+    from src a 
+    where b.value = a.value and a.key > '9'
+   ) 
+INSERT OVERWRITE TABLE src_5 
+  select *  
+  where b.key not in  ( select key from src s1 where s1.key > '2') 
+  order by key 
+;
+
+select * from src_4
+;
+select * from src_5
+;
+set hive.auto.convert.join=true;
+
+explain
+from src b 
+INSERT OVERWRITE TABLE src_4 
+  select * 
+  where b.key in 
+   (select a.key 
+    from src a 
+    where b.value = a.value and a.key > '9'
+   ) 
+INSERT OVERWRITE TABLE src_5 
+  select *  
+  where b.key not in  ( select key from src s1 where s1.key > '2') 
+  order by key 
+;
+
+from src b 
+INSERT OVERWRITE TABLE src_4 
+  select * 
+  where b.key in 
+   (select a.key 
+    from src a 
+    where b.value = a.value and a.key > '9'
+   ) 
+INSERT OVERWRITE TABLE src_5 
+  select *  
+  where b.key not in  ( select key from src s1 where s1.key > '2') 
+  order by key 
+;
+
+select * from src_4
+;
+select * from src_5
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists.q
new file mode 100644
index 0000000000000000000000000000000000000000..43a801fa9683e971e38d9bd3b1ebe5f1df5497a3
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists.q
@@ -0,0 +1,41 @@
+
+
+-- no agg, corr
+explain
+select * 
+from src b 
+where not exists 
+  (select a.key 
+  from src a 
+  where b.value = a.value  and a.key = b.key and a.value > 'val_2'
+  )
+;
+
+select * 
+from src b 
+where not exists 
+  (select a.key 
+  from src a 
+  where b.value = a.value  and a.key = b.key and a.value > 'val_2'
+  )
+;
+
+-- distinct, corr
+explain
+select * 
+from src b 
+where not exists 
+  (select distinct a.key 
+  from src a 
+  where b.value = a.value and a.value > 'val_2'
+  )
+;
+
+select * 
+from src b 
+where not exists 
+  (select a.key 
+  from src a 
+  where b.value = a.value and a.value > 'val_2'
+  )
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists_having.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists_having.q
new file mode 100644
index 0000000000000000000000000000000000000000..7205d17bc48618d7b30da2ee88e161ce780bd929
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists_having.q
@@ -0,0 +1,46 @@
+
+
+-- no agg, corr
+explain
+select * 
+from src b 
+group by key, value
+having not exists 
+  (select a.key 
+  from src a 
+  where b.value = a.value  and a.key = b.key and a.value > 'val_12'
+  )
+;
+
+select * 
+from src b 
+group by key, value
+having not exists 
+  (select a.key 
+  from src a 
+  where b.value = a.value  and a.key = b.key and a.value > 'val_12'
+  )
+;
+
+
+-- distinct, corr
+explain
+select * 
+from src b 
+group by key, value
+having not exists 
+  (select distinct a.key 
+  from src a 
+  where b.value = a.value and a.value > 'val_12'
+  )
+;
+
+select * 
+from src b 
+group by key, value
+having not exists 
+  (select distinct a.key 
+  from src a 
+  where b.value = a.value and a.value > 'val_12'
+  )
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin.q
new file mode 100644
index 0000000000000000000000000000000000000000..d5f60860313dae13ec03a440d958b0ae2f9f5ee9
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin.q
@@ -0,0 +1,143 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+DROP TABLE lineitem;
+CREATE TABLE lineitem (L_ORDERKEY      INT,
+                                L_PARTKEY       INT,
+                                L_SUPPKEY       INT,
+                                L_LINENUMBER    INT,
+                                L_QUANTITY      DOUBLE,
+                                L_EXTENDEDPRICE DOUBLE,
+                                L_DISCOUNT      DOUBLE,
+                                L_TAX           DOUBLE,
+                                L_RETURNFLAG    STRING,
+                                L_LINESTATUS    STRING,
+                                l_shipdate      STRING,
+                                L_COMMITDATE    STRING,
+                                L_RECEIPTDATE   STRING,
+                                L_SHIPINSTRUCT  STRING,
+                                L_SHIPMODE      STRING,
+                                L_COMMENT       STRING)
+ROW FORMAT DELIMITED
+FIELDS TERMINATED BY '|';
+
+LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem;
+
+-- non agg, non corr
+explain
+select * 
+from src 
+where src.key not in  
+  ( select key  from src s1 
+    where s1.key > '2'
+  )
+;
+
+select * 
+from src 
+where src.key not in  ( select key from src s1 where s1.key > '2')
+order by key
+;
+
+-- non agg, corr
+explain
+select p_mfgr, b.p_name, p_size 
+from part b 
+where b.p_name not in 
+  (select p_name 
+  from (select p_mfgr, p_name, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a 
+  where r <= 2 and b.p_mfgr = a.p_mfgr 
+  )
+;
+
+select p_mfgr, b.p_name, p_size 
+from part b 
+where b.p_name not in 
+  (select p_name 
+  from (select p_mfgr, p_name, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a 
+  where r <= 2 and b.p_mfgr = a.p_mfgr 
+  )
+order by p_mfgr, b.p_name
+;
+
+-- agg, non corr
+explain
+select p_name, p_size 
+from 
+part where part.p_size not in 
+  (select avg(p_size) 
+  from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a 
+  where r <= 2
+  )
+;
+select p_name, p_size 
+from 
+part where part.p_size not in 
+  (select avg(p_size) 
+  from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a 
+  where r <= 2
+  )
+order by p_name, p_size
+;
+
+-- agg, corr
+explain
+select p_mfgr, p_name, p_size 
+from part b where b.p_size not in 
+  (select min(p_size) 
+  from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a 
+  where r <= 2 and b.p_mfgr = a.p_mfgr
+  )
+;
+
+select p_mfgr, p_name, p_size 
+from part b where b.p_size not in 
+  (select min(p_size) 
+  from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a 
+  where r <= 2 and b.p_mfgr = a.p_mfgr
+  )
+order by p_mfgr, p_size
+;
+
+-- non agg, non corr, Group By in Parent Query
+select li.l_partkey, count(*) 
+from lineitem li 
+where li.l_linenumber = 1 and 
+  li.l_orderkey not in (select l_orderkey from lineitem where l_shipmode = 'AIR') 
+group by li.l_partkey
+;
+
+-- alternate not in syntax
+select * 
+from src 
+where not src.key in  ( select key from src s1 where s1.key > '2')
+order by key
+;
+
+-- null check
+create view T1_v as 
+select key from src where key <'11';
+
+create view T2_v as 
+select case when key > '104' then null else key end as key from T1_v;
+
+explain
+select * 
+from T1_v where T1_v.key not in (select T2_v.key from T2_v);
+
+select * 
+from T1_v where T1_v.key not in (select T2_v.key from T2_v);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin_having.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin_having.q
new file mode 100644
index 0000000000000000000000000000000000000000..a586f02272a739ae330f8aac9a96eb30e6837a4a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin_having.q
@@ -0,0 +1,74 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
+
+
+-- non agg, non corr
+explain
+select key, count(*) 
+from src 
+group by key
+having key not in  
+  ( select key  from src s1 
+    where s1.key > '12'
+  )
+;
+
+-- non agg, corr
+explain
+select b.p_mfgr, min(p_retailprice) 
+from part b 
+group by b.p_mfgr
+having b.p_mfgr not in 
+  (select p_mfgr 
+  from (select p_mfgr, min(p_retailprice) l, max(p_retailprice) r, avg(p_retailprice) a from part group by p_mfgr) a 
+  where min(p_retailprice) = l and r - l > 600
+  )
+;
+
+select b.p_mfgr, min(p_retailprice) 
+from part b 
+group by b.p_mfgr
+having b.p_mfgr not in 
+  (select p_mfgr 
+  from (select p_mfgr, min(p_retailprice) l, max(p_retailprice) r, avg(p_retailprice) a from part group by p_mfgr) a 
+  where min(p_retailprice) = l and r - l > 600
+  )
+;
+
+-- agg, non corr
+explain
+select b.p_mfgr, min(p_retailprice) 
+from part b 
+group by b.p_mfgr
+having b.p_mfgr not in 
+  (select p_mfgr 
+  from part a
+  group by p_mfgr
+  having max(p_retailprice) - min(p_retailprice) > 600
+  )
+;
+
+select b.p_mfgr, min(p_retailprice) 
+from part b 
+group by b.p_mfgr
+having b.p_mfgr not in 
+  (select p_mfgr 
+  from part a
+  group by p_mfgr
+  having max(p_retailprice) - min(p_retailprice) > 600
+  )
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_unqualcolumnrefs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_unqualcolumnrefs.q
new file mode 100644
index 0000000000000000000000000000000000000000..749435c0051348eba76c3fbad21c97f4990de3c0
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_unqualcolumnrefs.q
@@ -0,0 +1,83 @@
+DROP TABLE part;
+
+-- data setup
+CREATE TABLE part( 
+    p_partkey INT,
+    p_name STRING,
+    p_mfgr STRING,
+    p_brand STRING,
+    p_type STRING,
+    p_size INT,
+    p_container STRING,
+    p_retailprice DOUBLE,
+    p_comment STRING
+);
+
+create table src11 (key1 string, value1 string);
+
+create table part2( 
+    p2_partkey INT,
+    p2_name STRING,
+    p2_mfgr STRING,
+    p2_brand STRING,
+    p2_type STRING,
+    p2_size INT,
+    p2_container STRING,
+    p2_retailprice DOUBLE,
+    p2_comment STRING
+);
+
+-- non agg, corr
+explain select * from src11 where src11.key1 in (select key from src where src11.value1 = value and key > '9');
+
+explain select * from src a where a.key in (select key from src where a.value = value and key > '9');
+
+-- agg, corr
+explain
+select p_mfgr, p_name, p_size 
+from part b where b.p_size in 
+  (select min(p2_size) 
+    from (select p2_mfgr, p2_size, rank() over(partition by p2_mfgr order by p2_size) as r from part2) a 
+    where r <= 2 and b.p_mfgr = p2_mfgr
+  )
+;
+
+
+explain
+select p_mfgr, p_name, p_size 
+from part b where b.p_size in 
+  (select min(p_size) 
+   from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a 
+   where r <= 2 and b.p_mfgr = p_mfgr
+  )
+;
+
+-- distinct, corr
+explain 
+select * 
+from src b 
+where b.key in
+        (select distinct key 
+         from src 
+         where b.value = value and key > '9'
+        )
+;
+
+-- non agg, corr, having
+explain
+ select key, value, count(*) 
+from src b
+group by key, value
+having count(*) in (select count(*) from src where src.key > '9'  and src.value = b.value group by key )
+;
+
+-- non agg, corr
+explain
+select p_mfgr, b.p_name, p_size 
+from part b 
+where b.p_name not in 
+  (select p_name 
+  from (select p_mfgr, p_name, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a 
+  where r <= 2 and b.p_mfgr = p_mfgr 
+  )
+;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_views.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_views.q
new file mode 100644
index 0000000000000000000000000000000000000000..9f6712fc181ffe5b5dc08ed189c3dd9249e8cf67
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_views.q
@@ -0,0 +1,48 @@
+
+
+-- exists test
+create view cv1 as 
+select * 
+from src b 
+where exists
+  (select a.key 
+  from src a 
+  where b.value = a.value  and a.key = b.key and a.value > 'val_9')
+;
+
+select * 
+from cv1 where cv1.key in (select key from cv1 c where c.key > '95') order by key;
+;
+
+
+-- not in test
+create view cv2 as 
+select * 
+from src b 
+where b.key not in
+  (select a.key 
+  from src a 
+  where b.value = a.value  and a.key = b.key and a.value > 'val_11'
+  )
+;
+
+select * 
+from cv2 where cv2.key in (select key from cv2 c where c.key < '11') order by key;
+;
+
+-- in where + having
+create view cv3 as
+select key, value, count(*) 
+from src b
+where b.key in (select key from src where src.key > '8')
+group by key, value
+having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key )
+;
+
+select * from cv3 order by key;
+
+
+-- join of subquery views
+select *
+from cv3
+where cv3.key in (select key from cv1) order by key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q
index bb9d6f34ed8ecfbe550e292e2cd95d3d81241584..d633b97f4c931bd5a40e72dd86c5c94c2264ee5b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q
@@ -1,12 +1,12 @@
-
+DROP TABLE IF EXISTS symlink_text_input_format;
 
 EXPLAIN
 CREATE TABLE symlink_text_input_format (key STRING, value STRING) STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat';
 
 CREATE TABLE symlink_text_input_format (key STRING, value STRING) STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat';
 
-dfs -cp ../data/files/symlink1.txt ../build/ql/test/data/warehouse/symlink_text_input_format/symlink1.txt;
-dfs -cp ../data/files/symlink2.txt ../build/ql/test/data/warehouse/symlink_text_input_format/symlink2.txt;
+dfs -cp ../../data/files/symlink1.txt ${system:test.warehouse.dir}/symlink_text_input_format/symlink1.txt;
+dfs -cp ../../data/files/symlink2.txt ${system:test.warehouse.dir}/symlink_text_input_format/symlink2.txt;
 
 EXPLAIN SELECT * FROM symlink_text_input_format order by key, value;
 
@@ -20,5 +20,4 @@ EXPLAIN SELECT count(1) FROM symlink_text_input_format;
 
 SELECT count(1) FROM symlink_text_input_format;
 
-
-
+DROP TABLE symlink_text_input_format;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q
index 8b1a390149865582f77bf6d3bcbed27d6efb4fde..23209d85e4f15c17374569a64213eb2661f6c135 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q
@@ -4,7 +4,7 @@ SET hive.stats.collect.tablekeys=true;
 -- This test is used for testing the TableAccessAnalyzer
 
 CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
 
 CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE;
 CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q
index d2da5ac174a15e0b221de4e30ec38ae79321fb82..a4f0fdb1c1f26a8c3a9ace17bbb567d436593efe 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q
@@ -1,5 +1,5 @@
 create table if not exists test_boolean(dummy tinyint);
-insert overwrite table test_boolean  select 1 from src limit 1;
+insert overwrite table test_boolean  select 1 from src tablesample (1 rows);
 
 SELECT 1
 FROM (
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_dml.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_dml.q
new file mode 100644
index 0000000000000000000000000000000000000000..87d251f40fd7a2d9d33f9f6456c0484cbd5a6207
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_dml.q
@@ -0,0 +1,40 @@
+set hive.exec.dynamic.partition.mode=nonstrict;
+
+-- CTAS
+EXPLAIN CREATE TABLE tmp_src AS SELECT * FROM (SELECT value, count(value) AS cnt FROM src GROUP BY value) f1 ORDER BY cnt;
+CREATE TABLE tmp_src AS SELECT * FROM (SELECT value, count(value) AS cnt FROM src GROUP BY value) f1 ORDER BY cnt;
+
+SELECT * FROM tmp_src;
+
+-- dyn partitions
+CREATE TABLE tmp_src_part (c string) PARTITIONED BY (d int);
+EXPLAIN INSERT INTO TABLE tmp_src_part PARTITION (d) SELECT * FROM tmp_src;
+INSERT INTO TABLE tmp_src_part PARTITION (d) SELECT * FROM tmp_src;
+
+SELECT * FROM tmp_src_part;
+
+-- multi insert
+CREATE TABLE even (c int, d string);
+CREATE TABLE odd (c int, d string);
+
+EXPLAIN
+FROM src
+INSERT INTO TABLE even SELECT key, value WHERE key % 2 = 0 
+INSERT INTO TABLE odd SELECT key, value WHERE key % 2 = 1;
+
+FROM src
+INSERT INTO TABLE even SELECT key, value WHERE key % 2 = 0 
+INSERT INTO TABLE odd SELECT key, value WHERE key % 2 = 1;
+
+SELECT * FROM even;
+SELECT * FROM odd;
+
+-- create empty table
+CREATE TABLE empty STORED AS orc AS SELECT * FROM tmp_src_part WHERE d = -1000;
+SELECT * FROM empty;
+
+-- drop the tables
+DROP TABLE even;
+DROP TABLE odd;
+DROP TABLE tmp_src;
+DROP TABLE tmp_src_part;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_fsstat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_fsstat.q
new file mode 100644
index 0000000000000000000000000000000000000000..7f2e28fbafa8db6c79903eafae756d37ddfb7e72
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_fsstat.q
@@ -0,0 +1,19 @@
+set hive.execution.engine=tez;
+CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
+CREATE TABLE t1 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
+
+load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE t1 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE t1 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE t1 partition(ds='2008-04-08');
+load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE t1 partition(ds='2008-04-08');
+
+set hive.enforce.bucketing=true;
+set hive.enforce.sorting = true;
+set hive.optimize.bucketingsorting=false;
+set hive.stats.dbclass=fs;
+
+insert overwrite table tab_part partition (ds='2008-04-08')
+select key,value from t1;
+describe formatted tab_part partition(ds='2008-04-08');
+
+set hive.stats.dbclass=jdbc:derby;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_insert_overwrite_local_directory_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_insert_overwrite_local_directory_1.q
new file mode 100644
index 0000000000000000000000000000000000000000..d7a652fb8c8bd0b0b09deb1347a02e8b3f8baf8e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_insert_overwrite_local_directory_1.q
@@ -0,0 +1,5 @@
+insert overwrite local directory '${system:test.tmp.dir}/tez_local_src_table_1'
+select * from src order by key limit 10 ;
+dfs -cat file:${system:test.tmp.dir}/tez_local_src_table_1/000000_0 ;
+
+dfs -rmr file:${system:test.tmp.dir}/tez_local_src_table_1/ ;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_join_tests.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_join_tests.q
new file mode 100644
index 0000000000000000000000000000000000000000..f309e3fe0eed670b47a220cd938af31ca32d4237
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_join_tests.q
@@ -0,0 +1,12 @@
+explain
+select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key) order by b.key) x right outer join src c on (x.value = c.value) order by x.key;
+
+select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key) order by b.key) x right outer join src c on (x.value = c.value) order by x.key;
+select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key)) x right outer join src c on (x.value = c.value) order by x.key;
+select * from src1 a left outer join src b on (a.key = b.key) right outer join src c on (a.value = c.value) order by a.key;
+select * from src1 a left outer join src b on (a.key = b.key) left outer join src c on (a.value = c.value) order by a.key;
+select * from src1 a left outer join src b on (a.key = b.key) join src c on (a.key = c.key);
+select * from src1 a join src b on (a.key = b.key) join src c on (a.key = c.key);
+
+select count(*) from src1 a join src b on (a.key = b.key) join src c on (a.key = c.key);
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_joins_explain.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_joins_explain.q
new file mode 100644
index 0000000000000000000000000000000000000000..9193843824f6d083e90bfda9ad7326bd1b3217c5
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_joins_explain.q
@@ -0,0 +1,5 @@
+explain
+select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key) order by b.key) x right outer join src c on (x.value = c.value) order by x.key;
+
+select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key) order by b.key) x right outer join src c on (x.value = c.value) order by x.key;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_schema_evolution.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_schema_evolution.q
new file mode 100644
index 0000000000000000000000000000000000000000..2f1c73f8e528a59a713d1a6dbc3ec87906ee86cd
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_schema_evolution.q
@@ -0,0 +1,14 @@
+create table test (key int, value string) partitioned by (p int) stored as textfile;
+
+insert into table test partition (p=1) select * from src limit 10;
+
+alter table test set fileformat orc;
+
+insert into table test partition (p=2) select * from src limit 10;
+
+describe test;
+
+select * from test where p=1 and key > 0;
+select * from test where p=2 and key > 0;
+select * from test where key > 0;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_union.q
new file mode 100644
index 0000000000000000000000000000000000000000..f80d94c4a15fd1288ea6b0845b2896de438e0c23
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_union.q
@@ -0,0 +1,94 @@
+set hive.auto.convert.join=true;
+
+explain 
+select s1.key as key, s1.value as value from src s1 join src s3 on s1.key=s3.key
+UNION  ALL  
+select s2.key as key, s2.value as value from src s2;
+
+create table ut as
+select s1.key as key, s1.value as value from src s1 join src s3 on s1.key=s3.key
+UNION  ALL  
+select s2.key as key, s2.value as value from src s2;
+
+select * from ut order by key, value limit 20;
+drop table ut;
+
+set hive.auto.convert.join=false;
+
+explain
+with u as (select * from src union all select * from src)
+select count(*) from (select u1.key as k1, u2.key as k2 from
+u as u1 join u as u2 on (u1.key = u2.key)) a;
+
+create table ut as
+with u as (select * from src union all select * from src)
+select count(*) as cnt from (select u1.key as k1, u2.key as k2 from
+u as u1 join u as u2 on (u1.key = u2.key)) a;
+
+select * from ut order by cnt limit 20;
+drop table ut;
+
+set hive.auto.convert.join=true;
+
+explain select s1.key as skey, u1.key as ukey from
+src s1
+join (select * from src union all select * from src) u1 on s1.key = u1.key;
+
+create table ut as
+select s1.key as skey, u1.key as ukey from
+src s1
+join (select * from src union all select * from src) u1 on s1.key = u1.key;
+
+select * from ut order by skey, ukey limit 20;
+drop table ut;
+
+explain select s1.key as skey, u1.key as ukey, s8.key as lkey from 
+src s1
+join (select s2.key as key from src s2 join src s3 on s2.key = s3.key
+      union all select s4.key from src s4 join src s5 on s4.key = s5.key
+      union all select s6.key from src s6 join src s7 on s6.key = s7.key) u1 on (s1.key = u1.key)
+join src s8 on (u1.key = s8.key)
+order by lkey;
+
+create table ut as
+select s1.key as skey, u1.key as ukey, s8.key as lkey from 
+src s1
+join (select s2.key as key from src s2 join src s3 on s2.key = s3.key
+      union all select s4.key from src s4 join src s5 on s4.key = s5.key
+      union all select s6.key from src s6 join src s7 on s6.key = s7.key) u1 on (s1.key = u1.key)
+join src s8 on (u1.key = s8.key)
+order by lkey;
+
+select * from ut order by skey, ukey, lkey limit 100;
+
+drop table ut;
+
+explain
+select s2.key as key from src s2 join src s3 on s2.key = s3.key
+union all select s4.key from src s4 join src s5 on s4.key = s5.key;
+
+create table ut as
+select s2.key as key from src s2 join src s3 on s2.key = s3.key
+union all select s4.key from src s4 join src s5 on s4.key = s5.key;
+
+select * from ut order by key limit 30;
+
+drop table ut;
+
+explain
+select * from
+(select * from src union all select * from src) u
+left outer join src s on u.key = s.key;
+
+explain
+select u.key as ukey, s.key as skey from
+(select * from src union all select * from src) u
+right outer join src s on u.key = s.key;
+
+create table ut as
+select u.key as ukey, s.key as skey from
+(select * from src union all select * from src) u
+right outer join src s on u.key = s.key;
+
+select * from ut order by ukey, skey limit 20;
+drop table ut;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q
index f2c3b596af44ac201fca48d68310f312023f773e..ce79eefaae4d37bcc1db6b220ec06fe0664810d9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q
@@ -1,10 +1,12 @@
+set hive.fetch.task.conversion=more;
+
 drop table timestamp_1;
 
 create table timestamp_1 (t timestamp);
 alter table timestamp_1 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe';
 
-insert overwrite table timestamp_1 
-  select cast('2011-01-01 01:01:01' as timestamp) from src limit 1;
+insert overwrite table timestamp_1
+  select cast('2011-01-01 01:01:01' as timestamp) from src tablesample (1 rows);
 select cast(t as boolean) from timestamp_1 limit 1;
 select cast(t as tinyint) from timestamp_1 limit 1;
 select cast(t as smallint) from timestamp_1 limit 1;
@@ -15,7 +17,7 @@ select cast(t as double) from timestamp_1 limit 1;
 select cast(t as string) from timestamp_1 limit 1;
 
 insert overwrite table timestamp_1
-  select '2011-01-01 01:01:01' from src limit 1;
+  select '2011-01-01 01:01:01' from src tablesample (1 rows);
 select cast(t as boolean) from timestamp_1 limit 1;
 select cast(t as tinyint) from timestamp_1 limit 1;
 select cast(t as smallint) from timestamp_1 limit 1;
@@ -26,7 +28,7 @@ select cast(t as double) from timestamp_1 limit 1;
 select cast(t as string) from timestamp_1 limit 1;
 
 insert overwrite table timestamp_1
-  select '2011-01-01 01:01:01.1' from src limit 1;
+  select '2011-01-01 01:01:01.1' from src tablesample (1 rows);
 select cast(t as boolean) from timestamp_1 limit 1;
 select cast(t as tinyint) from timestamp_1 limit 1;
 select cast(t as smallint) from timestamp_1 limit 1;
@@ -37,7 +39,7 @@ select cast(t as double) from timestamp_1 limit 1;
 select cast(t as string) from timestamp_1 limit 1;
 
 insert overwrite table timestamp_1
-  select '2011-01-01 01:01:01.0001' from src limit 1;
+  select '2011-01-01 01:01:01.0001' from src tablesample (1 rows);
 select cast(t as boolean) from timestamp_1 limit 1;
 select cast(t as tinyint) from timestamp_1 limit 1;
 select cast(t as smallint) from timestamp_1 limit 1;
@@ -48,7 +50,7 @@ select cast(t as double) from timestamp_1 limit 1;
 select cast(t as string) from timestamp_1 limit 1;
 
 insert overwrite table timestamp_1
-  select '2011-01-01 01:01:01.000100000' from src limit 1;
+  select '2011-01-01 01:01:01.000100000' from src tablesample (1 rows);
 select cast(t as boolean) from timestamp_1 limit 1;
 select cast(t as tinyint) from timestamp_1 limit 1;
 select cast(t as smallint) from timestamp_1 limit 1;
@@ -59,7 +61,7 @@ select cast(t as double) from timestamp_1 limit 1;
 select cast(t as string) from timestamp_1 limit 1;
 
 insert overwrite table timestamp_1
-  select '2011-01-01 01:01:01.001000011' from src limit 1;
+  select '2011-01-01 01:01:01.001000011' from src tablesample (1 rows);
 select cast(t as boolean) from timestamp_1 limit 1;
 select cast(t as tinyint) from timestamp_1 limit 1;
 select cast(t as smallint) from timestamp_1 limit 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q
index b93208f48c453e0dac5d868d083b960c64fd97a6..351f5ca519499e130eac64fc3dc8957d6b1085ce 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q
@@ -1,10 +1,12 @@
+set hive.fetch.task.conversion=more;
+
 drop table timestamp_2;
 
 create table timestamp_2 (t timestamp);
 alter table timestamp_2 set serde 'org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe';
 
-insert overwrite table timestamp_2 
-  select cast('2011-01-01 01:01:01' as timestamp) from src limit 1;
+insert overwrite table timestamp_2
+  select cast('2011-01-01 01:01:01' as timestamp) from src tablesample (1 rows);
 select cast(t as boolean) from timestamp_2 limit 1;
 select cast(t as tinyint) from timestamp_2 limit 1;
 select cast(t as smallint) from timestamp_2 limit 1;
@@ -15,7 +17,7 @@ select cast(t as double) from timestamp_2 limit 1;
 select cast(t as string) from timestamp_2 limit 1;
 
 insert overwrite table timestamp_2
-  select '2011-01-01 01:01:01' from src limit 1;
+  select '2011-01-01 01:01:01' from src tablesample (1 rows);
 select cast(t as boolean) from timestamp_2 limit 1;
 select cast(t as tinyint) from timestamp_2 limit 1;
 select cast(t as smallint) from timestamp_2 limit 1;
@@ -26,7 +28,7 @@ select cast(t as double) from timestamp_2 limit 1;
 select cast(t as string) from timestamp_2 limit 1;
 
 insert overwrite table timestamp_2
-  select '2011-01-01 01:01:01.1' from src limit 1;
+  select '2011-01-01 01:01:01.1' from src tablesample (1 rows);
 select cast(t as boolean) from timestamp_2 limit 1;
 select cast(t as tinyint) from timestamp_2 limit 1;
 select cast(t as smallint) from timestamp_2 limit 1;
@@ -37,7 +39,7 @@ select cast(t as double) from timestamp_2 limit 1;
 select cast(t as string) from timestamp_2 limit 1;
 
 insert overwrite table timestamp_2
-  select '2011-01-01 01:01:01.0001' from src limit 1;
+  select '2011-01-01 01:01:01.0001' from src tablesample (1 rows);
 select cast(t as boolean) from timestamp_2 limit 1;
 select cast(t as tinyint) from timestamp_2 limit 1;
 select cast(t as smallint) from timestamp_2 limit 1;
@@ -48,7 +50,7 @@ select cast(t as double) from timestamp_2 limit 1;
 select cast(t as string) from timestamp_2 limit 1;
 
 insert overwrite table timestamp_2
-  select '2011-01-01 01:01:01.000100000' from src limit 1;
+  select '2011-01-01 01:01:01.000100000' from src tablesample (1 rows);
 select cast(t as boolean) from timestamp_2 limit 1;
 select cast(t as tinyint) from timestamp_2 limit 1;
 select cast(t as smallint) from timestamp_2 limit 1;
@@ -59,7 +61,7 @@ select cast(t as double) from timestamp_2 limit 1;
 select cast(t as string) from timestamp_2 limit 1;
 
 insert overwrite table timestamp_2
-  select '2011-01-01 01:01:01.001000011' from src limit 1;
+  select '2011-01-01 01:01:01.001000011' from src tablesample (1 rows);
 select cast(t as boolean) from timestamp_2 limit 1;
 select cast(t as tinyint) from timestamp_2 limit 1;
 select cast(t as smallint) from timestamp_2 limit 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q
index cda724f9e8f43aee13ee78cabbe70e1a4c3fb2b5..0e1a8d552678349b8696da717c371c2083cec042 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q
@@ -1,10 +1,12 @@
+set hive.fetch.task.conversion=more;
+
 drop table timestamp_3;
 
 create table timestamp_3 (t timestamp);
 alter table timestamp_3 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe';
 
-insert overwrite table timestamp_3 
-  select cast(cast('1.3041352164485E9' as double) as timestamp) from src limit 1;
+insert overwrite table timestamp_3
+  select cast(cast('1.3041352164485E9' as double) as timestamp) from src tablesample (1 rows);
 select cast(t as boolean) from timestamp_3 limit 1;
 select cast(t as tinyint) from timestamp_3 limit 1;
 select cast(t as smallint) from timestamp_3 limit 1;
@@ -14,4 +16,6 @@ select cast(t as float) from timestamp_3 limit 1;
 select cast(t as double) from timestamp_3 limit 1;
 select cast(t as string) from timestamp_3 limit 1;
 
+select t, sum(t), count(*), sum(t)/count(*), avg(t) from timestamp_3 group by t;
+
 drop table timestamp_3;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q
index f64ae48b85db70e615a1d471cb49e4eb284f8d8b..30fee3cbf6013eb93c135afda52f235da2909fdc 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q
@@ -1,5 +1,6 @@
+set hive.fetch.task.conversion=more;
 
-select cast('2011-05-06 07:08:09' as timestamp) > 
+select cast('2011-05-06 07:08:09' as timestamp) >
   cast('2011-05-06 07:08:09' as timestamp) from src limit 1;
 
 select cast('2011-05-06 07:08:09' as timestamp) <
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q
index 7a1005295eb280e42d4babf5ed0f7ff31019ad47..e9a0cfae7cc9acce052b27f4d1199c60fb71332e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q
@@ -1,6 +1,6 @@
 drop table timestamp_lazy;
 create table timestamp_lazy (t timestamp, key string, value string);
-insert overwrite table timestamp_lazy select cast('2011-01-01 01:01:01' as timestamp), key, value from src limit 5;
+insert overwrite table timestamp_lazy select cast('2011-01-01 01:01:01' as timestamp), key, value from src tablesample (5 rows);
 
 select t,key,value from timestamp_lazy ORDER BY key ASC, value ASC;
 select t,key,value from timestamp_lazy distribute by t sort by key ASC, value ASC;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q
index efd5bc4b78bf419783734a8e1872b09297d9376c..36f35413e9b482f26930ba5d19499383cab025ec 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q
@@ -1,6 +1,6 @@
 DROP TABLE IF EXISTS timestamp_null;
 CREATE TABLE timestamp_null (t1 TIMESTAMP);
-LOAD DATA LOCAL INPATH '../data/files/test.dat' OVERWRITE INTO TABLE timestamp_null;
+LOAD DATA LOCAL INPATH '../../data/files/test.dat' OVERWRITE INTO TABLE timestamp_null;
 
 SELECT * FROM timestamp_null LIMIT 1;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q
index 2620acefee7f773b32cc4044d122b7eec2c303df..ade9fb408c0923407ea13f138a240a9a8c701bf4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q
@@ -1,13 +1,15 @@
+set hive.fetch.task.conversion=more;
+
 drop table timestamp_udf;
 drop table timestamp_udf_string;
 
 create table timestamp_udf (t timestamp);
 create table timestamp_udf_string (t string);
-from src
+from (select * from src tablesample (1 rows)) s
   insert overwrite table timestamp_udf 
-    select '2011-05-06 07:08:09.1234567' limit 1
+    select '2011-05-06 07:08:09.1234567'
   insert overwrite table timestamp_udf_string
-    select '2011-05-06 07:08:09.1234567' limit 1;
+    select '2011-05-06 07:08:09.1234567';
 
 -- Test UDFs with Timestamp input
 select unix_timestamp(t), year(t), month(t), day(t), dayofmonth(t),
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q
index 962077c2ca565d5453d89caf4d4ebea8fab5bbda..3bed2b6727e7b19df4ef0ab515de4c0733ea6af2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q
@@ -12,7 +12,7 @@ SELECT transform(*) USING 'cat' AS (col array<bigint>) FROM transform1_t1;
 create table transform1_t2(col array<int>);
 
 insert overwrite table transform1_t2
-select array(1,2,3) from src limit 1;
+select array(1,2,3) from src tablesample (1 rows);
 
 EXPLAIN
 SELECT transform('0\0021\0022') USING 'cat' AS (col array<int>) FROM transform1_t2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q
index d756b47e464d0954d0e9e8579e448762efdf595e..0bfb23ead68691221fe28c486214bc01f85a4343 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q
@@ -5,7 +5,7 @@ ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' STORED A
 
 set hive.stats.autogather=true;
 
-INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 10;
+INSERT OVERWRITE TABLE test_tab SELECT * FROM src tablesample (10 rows);
 
 DESC FORMATTED test_tab;
 
@@ -20,7 +20,7 @@ DESC FORMATTED test_tab;
 SELECT * FROM test_tab ORDER BY value;
 
 -- Truncate multiple columns
-INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 10;
+INSERT OVERWRITE TABLE test_tab SELECT * FROM src tablesample (10 rows);
 
 TRUNCATE TABLE test_tab COLUMNS (key, value);
 
@@ -40,7 +40,7 @@ SELECT * FROM test_tab ORDER BY value;
 -- Test truncating with a binary serde
 ALTER TABLE test_tab SET SERDE 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe';
 
-INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 10;
+INSERT OVERWRITE TABLE test_tab SELECT * FROM src tablesample (10 rows);
 
 DESC FORMATTED test_tab;
 
@@ -65,7 +65,7 @@ SELECT * FROM test_tab ORDER BY value;
 -- Test truncating a partition
 CREATE TABLE test_tab_part (key STRING, value STRING) PARTITIONED BY (part STRING) STORED AS RCFILE;
 
-INSERT OVERWRITE TABLE test_tab_part PARTITION (part = '1') SELECT * FROM src LIMIT 10;
+INSERT OVERWRITE TABLE test_tab_part PARTITION (part = '1') SELECT * FROM src tablesample (10 rows);
 
 DESC FORMATTED test_tab_part PARTITION (part = '1');
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q
index a7aab357eaf2f9b65d39d0dcdc1e6a6c19df2bde..7a59efc4d1b0146c7f212415edbe37ce4b3d62ec 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q
@@ -2,9 +2,9 @@
 
 CREATE TABLE test_tab (key STRING, value STRING) STORED AS RCFILE;
 
-INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 5;
+INSERT OVERWRITE TABLE test_tab SELECT * FROM src tablesample (5 rows);
 
-INSERT INTO TABLE test_tab SELECT * FROM src LIMIT 5;
+INSERT INTO TABLE test_tab SELECT * FROM src tablesample (5 rows);
 
 -- The value should be 2 indicating the table has 2 files
 SELECT COUNT(DISTINCT INPUT__FILE__NAME) FROM test_tab;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q
index c0e81e9ac051fb1807d8ef0efb407fc7c9661570..975c0f1ae8426de8752d33195ce6339bdeef4519 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q
@@ -1,5 +1,5 @@
 create table src_truncate (key string, value string);
-load data local inpath '../data/files/kv1.txt' into table src_truncate;;
+load data local inpath '../../data/files/kv1.txt' into table src_truncate;;
 
 create table srcpart_truncate (key string, value string) partitioned by (ds string, hr string);
 alter table srcpart_truncate add partition (ds='2008-04-08', hr='11');        
@@ -7,10 +7,10 @@ alter table srcpart_truncate add partition (ds='2008-04-08', hr='12');
 alter table srcpart_truncate add partition (ds='2008-04-09', hr='11');
 alter table srcpart_truncate add partition (ds='2008-04-09', hr='12');
 
-load data local inpath '../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-08', hr='11');
-load data local inpath '../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-08', hr='12');
-load data local inpath '../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-09', hr='11');
-load data local inpath '../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-09', hr='12');
+load data local inpath '../../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-08', hr='11');
+load data local inpath '../../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-08', hr='12');
+load data local inpath '../../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-09', hr='11');
+load data local inpath '../../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-09', hr='12');
 
 set hive.fetch.task.convertion=more;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q
index a1607320c7a2d7dc6a180cd112b83cc91ca2099d..4d1d978f829eef415acc9830a2417504601fd99c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 EXPLAIN
 SELECT IF(false, 1, cast(2 as smallint)) + 3 FROM src LIMIT 1;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q
index 63dd66ebedd0c121015880b87c44279577a90ca4..4c4a828fe0ee236c5170e3477c68641eb35e6401 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q
@@ -1,3 +1,4 @@
+set hive.fetch.task.conversion=more;
 
 -- casting from null should yield null
 select
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q
index 0d36bc44fe08bca2de95375dd6130508574c2855..b18c01425c8c6148a15e024954c750638d9824eb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q
@@ -1,3 +1,4 @@
+set hive.fetch.task.conversion=more;
 -- Check for int, bigint automatic type widening conversions in UDFs, UNIONS
 EXPLAIN SELECT COALESCE(0, 9223372036854775807) FROM src LIMIT 1;
 SELECT COALESCE(0, 9223372036854775807) FROM src LIMIT 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q
index 45aaa022dc8a4024e82a67fde308a74b9cbf56ce..04bea32101bdf92ae51bd2108f247c2cea59cb42 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q
@@ -1,6 +1,9 @@
 DESCRIBE FUNCTION collect_set;
 DESCRIBE FUNCTION EXTENDED collect_set;
 
+DESCRIBE FUNCTION collect_list;
+DESCRIBE FUNCTION EXTENDED collect_list;
+
 set hive.map.aggr = false;
 set hive.groupby.skewindata = false;
 
@@ -8,6 +11,10 @@ SELECT key, collect_set(value)
 FROM src
 GROUP BY key ORDER BY key limit 20;
 
+SELECT key, collect_list(value)
+FROM src
+GROUP BY key ORDER by key limit 20;
+
 set hive.map.aggr = true;
 set hive.groupby.skewindata = false;
 
@@ -15,6 +22,10 @@ SELECT key, collect_set(value)
 FROM src
 GROUP BY key ORDER BY key limit 20;
 
+SELECT key, collect_list(value)
+FROM src
+GROUP BY key ORDER BY key limit 20;
+
 set hive.map.aggr = false;
 set hive.groupby.skewindata = true;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q
index dda7aaa5f20410a24e8b45675abc1b390bfed84d..f065385688a1d5f78469e80c8355ca520b491719 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q
@@ -1,5 +1,5 @@
 CREATE TABLE kafka (contents STRING);
-LOAD DATA LOCAL INPATH '../data/files/text-en.txt' INTO TABLE kafka;
+LOAD DATA LOCAL INPATH '../../data/files/text-en.txt' INTO TABLE kafka;
 set mapred.reduce.tasks=1;
 set hive.exec.reducers.max=1;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q
index 6cc9ce2630dd1c21ddd51c018af6d5ab6ef48114..a2edec4d64e4ccc39783b362693fec597f4b01a0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q
@@ -2,7 +2,7 @@ DROP TABLE covar_tab;
 CREATE TABLE covar_tab (a INT, b INT, c INT)
 ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'
 STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/covar_tab.txt' OVERWRITE
+LOAD DATA LOCAL INPATH '../../data/files/covar_tab.txt' OVERWRITE
 INTO TABLE covar_tab;
 
 DESCRIBE FUNCTION corr;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q
index 0f5d5f35bf02a7c94723779af8c98c4ccdb0fc81..a9937bae3c21cce739ed390a3caee4a018612c2e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q
@@ -2,7 +2,7 @@ DROP TABLE covar_tab;
 CREATE TABLE covar_tab (a INT, b INT, c INT)
 ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'
 STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/covar_tab.txt' OVERWRITE
+LOAD DATA LOCAL INPATH '../../data/files/covar_tab.txt' OVERWRITE
 INTO TABLE covar_tab;
 
 DESCRIBE FUNCTION covar_pop;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q
index 72b9c4bd400491391a84d99d1ff577faf1edd4b2..2b50d8f23814094c96de76b42f30d66f67780fa5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q
@@ -2,7 +2,7 @@ DROP TABLE covar_tab;
 CREATE TABLE covar_tab (a INT, b INT, c INT)
 ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'
 STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/covar_tab.txt' OVERWRITE
+LOAD DATA LOCAL INPATH '../../data/files/covar_tab.txt' OVERWRITE
 INTO TABLE covar_tab;
 
 DESCRIBE FUNCTION covar_samp;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q
index 31ffd29a88acd4cdf45521016f120cd20e0033e7..6a2fde52e42f610d764c07f55b9a19ab32199267 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q
@@ -1,5 +1,5 @@
 CREATE TABLE kafka (contents STRING);
-LOAD DATA LOCAL INPATH '../data/files/text-en.txt' INTO TABLE kafka;
+LOAD DATA LOCAL INPATH '../../data/files/text-en.txt' INTO TABLE kafka;
 set mapred.reduce.tasks=1;
 set hive.exec.reducers.max=1;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile.q
new file mode 100644
index 0000000000000000000000000000000000000000..8ebf01dcecb8c38262b6b77a42990b733ff47139
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile.q
@@ -0,0 +1 @@
+select percentile(cast(key as bigint), 0.3) from src;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q
index 66c408d71bc1a8f005af0d901c453dec8b29e693..5b8ad7a08f5d38ff86a16d3e7b2e43cbe3e66796 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q
@@ -1,10 +1,10 @@
 -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S)
 
 CREATE TABLE bucket (key double, value string) CLUSTERED BY (key) SORTED BY (key DESC)  INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket;
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket;
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket;
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket;
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket;
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket;
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket;
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket;
 
 create table t1 (result double);
 create table t2 (result double);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q
index 07bfb6e1fb2ab1541e53e3d257b091389257a61e..1efa2951efd27ec4eddefbf93af908ac8a9b1d32 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q
@@ -2,10 +2,10 @@
 -- 0.23 changed input order of data in reducer task, which affects result of percentile_approx
 
 CREATE TABLE bucket (key double, value string) CLUSTERED BY (key) SORTED BY (key DESC)  INTO 4 BUCKETS STORED AS TEXTFILE;
-load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket;
-load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket;
-load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket;
-load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket;
+load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket;
+load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket;
+load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket;
+load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket;
 
 create table t1 (result double);
 create table t2 (result double);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_sum_list.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_sum_list.q
new file mode 100644
index 0000000000000000000000000000000000000000..0d86a42128d2502740ad9a7e6289051e3efc6c85
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_sum_list.q
@@ -0,0 +1,6 @@
+-- HIVE-5279
+-- GenericUDAFSumList has Converter which does not have default constructor
+-- After
+create temporary function sum_list as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSumList';
+
+select sum_list(array(key, key)) from src;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q
index 113af61062b0d5fbf46b98983fe72226a3c71c5f..41bdec08278a8bf04f8fd7d0bafd54f2637b65ea 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q
@@ -1,14 +1,16 @@
-explain 
-select E() FROM src LIMIT 1;
+set hive.fetch.task.conversion=more;
+
+explain
+select E() FROM src tablesample (1 rows);
 
-select E() FROM src LIMIT 1;
+select E() FROM src tablesample (1 rows);
 
 DESCRIBE FUNCTION E;
 DESCRIBE FUNCTION EXTENDED E;
 explain 
-select E() FROM src LIMIT 1;
+select E() FROM src tablesample (1 rows);
 
-select E() FROM src LIMIT 1;
+select E() FROM src tablesample (1 rows);
 
 DESCRIBE FUNCTION E;
 DESCRIBE FUNCTION EXTENDED E;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q
index 1fde7df5d251e6092ae63163f259dcc683f6eb5e..945483ecbfea8cc7beb13ef6e4d9d3c40b94cf22 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q
@@ -1,14 +1,16 @@
-explain 
-select PI() FROM src LIMIT 1;
+set hive.fetch.task.conversion=more;
+
+explain
+select PI() FROM src tablesample (1 rows);
 
-select PI() FROM src LIMIT 1;
+select PI() FROM src tablesample (1 rows);
 
 DESCRIBE FUNCTION PI;
 DESCRIBE FUNCTION EXTENDED PI;
 explain 
-select PI() FROM src LIMIT 1;
+select PI() FROM src tablesample (1 rows);
 
-select PI() FROM src LIMIT 1;
+select PI() FROM src tablesample (1 rows);
 
 DESCRIBE FUNCTION PI;
 DESCRIBE FUNCTION EXTENDED PI;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q
index f4f227d0dc1688f5ca97201b645d19b35d9a082a..0c06a5b6cd7f961a0709d4b376a437e2f6a4e14d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION abs;
 DESCRIBE FUNCTION EXTENDED abs;
 
@@ -7,7 +9,7 @@ EXPLAIN SELECT
   abs(123),
   abs(-9223372036854775807),
   abs(9223372036854775807)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   abs(0),
@@ -15,16 +17,16 @@ SELECT
   abs(123),
   abs(-9223372036854775807),
   abs(9223372036854775807)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 EXPLAIN SELECT
   abs(0.0),
   abs(-3.14159265),
   abs(3.14159265)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   abs(0.0),
   abs(-3.14159265),
   abs(3.14159265)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q
index 625a2aa5c6aa6ebc94d9c272f6c0fc25cc24ca1a..f9adc16931dd60fa5003b7aa34d7ae83cfbd1260 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q
@@ -1,14 +1,16 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION acos;
 DESCRIBE FUNCTION EXTENDED acos;
 
 SELECT acos(null)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT acos(0)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT acos(-0.5), asin(0.66)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT acos(2)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q
index fca8fe8d1c00c0db306b6000a07be4a7051d4855..5a6a1830b1a51e2f4442f61b71b061ac8f1f21d7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q
@@ -1,8 +1,10 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION array;
 DESCRIBE FUNCTION EXTENDED array;
 
 EXPLAIN SELECT array(), array()[1], array(1, 2, 3), array(1, 2, 3)[2], array(1,"a", 2, 3), array(1,"a", 2, 3)[2],
-array(array(1), array(2), array(3), array(4))[1][0] FROM src LIMIT 1;
+array(array(1), array(2), array(3), array(4))[1][0] FROM src tablesample (1 rows);
 
 SELECT array(), array()[1], array(1, 2, 3), array(1, 2, 3)[2], array(1,"a", 2, 3), array(1,"a", 2, 3)[2],
-array(array(1), array(2), array(3), array(4))[1][0] FROM src LIMIT 1;
+array(array(1), array(2), array(3), array(4))[1][0] FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q
index 937bb0be03bd3f07670c945176324a423ae3b19a..d2dad644065a2d2bd04d49c215e24ed9bfe1cc2d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q
@@ -1,9 +1,11 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION array_contains;
 DESCRIBE FUNCTION EXTENDED array_contains;
 
 -- evalutes function for array of primitives
-SELECT array_contains(array(1, 2, 3), 1) FROM src LIMIT 1;
+SELECT array_contains(array(1, 2, 3), 1) FROM src tablesample (1 rows);
 
 -- evaluates function for nested arrays
 SELECT array_contains(array(array(1,2), array(2,3), array(3,4)), array(1,2))
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q
index 53b389fd38fd8ef7f4544f99d7e336e4fc45c9e8..3d885a2563520e60dc5e4c20e62a89635d0af970 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION ascii;
 DESCRIBE FUNCTION EXTENDED ascii;
 
@@ -5,10 +7,10 @@ EXPLAIN SELECT
   ascii('Facebook'),
   ascii(''),
   ascii('!')
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   ascii('Facebook'),
   ascii(''),
   ascii('!')
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q
index f95a5f57df8f813e007b4e2ecece420314b9eec6..73b77d10f0b692f5d7ff0c61dfdb42de7b470793 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q
@@ -1,14 +1,16 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION asin;
 DESCRIBE FUNCTION EXTENDED asin;
 
 SELECT asin(null)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT asin(0)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT asin(-0.5), asin(0.66)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT asin(2)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q
index d4ef03deb81b0ee1c159b82f04afeb5351ee4a0a..090438cb0f0cb89b38edaff79ddf772480afb22d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q
@@ -1,16 +1,18 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION atan;
 DESCRIBE FUNCTION EXTENDED atan;
 
 SELECT atan(null)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT atan(1), atan(6), atan(-1.0)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 DESCRIBE FUNCTION atan;
 DESCRIBE FUNCTION EXTENDED atan;
 
 SELECT atan(null)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT atan(1), atan(6), atan(-1.0)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q
index eb3ccea82e631a602eb62be9c529bec84e78be83..b22ee9c3cecf1f98b4104cfcb8f92294123b2e6a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 describe function between;
 describe function extended between;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q
index 2b9ad62a39dbeeb65f2f157a4d31b072cf81611f..c5a7ac1a60bcd1a5d333bcc17938517c3001fffc 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION bin;
 DESCRIBE FUNCTION EXTENDED bin;
 
@@ -5,7 +7,7 @@ SELECT
   bin(1),
   bin(0),
   bin(99992421)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 -- Negative numbers should be treated as two's complement (64 bit).
-SELECT bin(-5) FROM src LIMIT 1;
+SELECT bin(-5) FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q
index 7ea50dac31d07e1399f85ba5656bfc55c93c667d..ed7711cd6d5de3978b17ed2fa7a5ca5dbc33a4d9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q
@@ -1,11 +1,13 @@
-select ewah_bitmap_and(array(13,2,4,8589934592,4096,0), array(13,2,4,8589934592,4096,0)) from src limit 1;
-select ewah_bitmap_and(array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0)) from src limit 1;
+set hive.fetch.task.conversion=more;
+
+select ewah_bitmap_and(array(13,2,4,8589934592,4096,0), array(13,2,4,8589934592,4096,0)) from src tablesample (1 rows);
+select ewah_bitmap_and(array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0)) from src tablesample (1 rows);
 
 drop table bitmap_test;
 create table bitmap_test (a array<bigint>, b array<bigint>);
 
 insert overwrite table bitmap_test
-select array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0) from src limit 10;
+select array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0) from src tablesample (10 rows);
 
 select ewah_bitmap_and(a,b) from bitmap_test;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q
index 88e961683e2892f29f8d25b4291ba3cadee4fb57..142b248cdd2504aca7e071bf2975038b1ba58f4d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q
@@ -1,3 +1,5 @@
-select ewah_bitmap_empty(array(13,2,4,8589934592,0,0)) from src limit 1;
+set hive.fetch.task.conversion=more;
 
-select ewah_bitmap_empty(array(13,2,4,8589934592,4096,0)) from src limit 1;
+select ewah_bitmap_empty(array(13,2,4,8589934592,0,0)) from src tablesample (1 rows);
+
+select ewah_bitmap_empty(array(13,2,4,8589934592,4096,0)) from src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q
index 0b71e681a53d5d1d41a928a0df4eae0767f3fff7..00785b73faa050a950e0c8af6fa58977f6d464c6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q
@@ -1,11 +1,13 @@
-select ewah_bitmap_or(array(13,2,4,8589934592,4096,0), array(13,2,4,8589934592,4096,0)) from src limit 1;
-select ewah_bitmap_or(array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0)) from src limit 1;
+set hive.fetch.task.conversion=more;
+
+select ewah_bitmap_or(array(13,2,4,8589934592,4096,0), array(13,2,4,8589934592,4096,0)) from src tablesample (1 rows);
+select ewah_bitmap_or(array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0)) from src tablesample (1 rows);
 
 drop table bitmap_test;
 create table bitmap_test (a array<bigint>, b array<bigint>);
 
 insert overwrite table bitmap_test
-select array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0) from src limit 10;
+select array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0) from src tablesample (10 rows);
 
 select ewah_bitmap_or(a,b) from bitmap_test;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q
index 4f71e70e1f3bf3705fd160cc7abf08383f3d6e59..43573bfb12fd8adfeb6244250bf3bc99d6a07563 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION case;
 DESCRIBE FUNCTION EXTENDED case;
 
@@ -27,7 +29,7 @@ SELECT CASE 1
         WHEN 22 THEN 23
         WHEN 21 THEN 24
        END
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT CASE 1
         WHEN 1 THEN 2
@@ -54,10 +56,27 @@ SELECT CASE 1
         WHEN 22 THEN 23
         WHEN 21 THEN 24
        END
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 -- verify that short-circuiting is working correctly for CASE
 -- we should never get to the ELSE branch, which would raise an exception
 SELECT CASE 1 WHEN 1 THEN 'yo'
 ELSE reflect('java.lang.String', 'bogus', 1) END
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
+
+-- Allow compatible types in when/return type
+SELECT CASE 1
+        WHEN 1 THEN 123.0BD
+        ELSE 0.0BD
+       END,
+       CASE 1
+        WHEN 1.0 THEN 123
+        WHEN 2 THEN 1.0
+        ELSE 222.02BD
+       END,
+       CASE 'abc'
+        WHEN cast('abc' as varchar(3)) THEN 'abcd'
+        WHEN 'efg' THEN cast('efgh' as varchar(10))
+        ELSE cast('ijkl' as char(4))
+       END
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q
index 736bb053cddc7a924f8ef98446d3157de4080b91..2aa76f1f1d82e13850bcf716ca1092677d61f294 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 EXPLAIN
 SELECT CASE src_thrift.lint[0]
         WHEN 0 THEN src_thrift.lint[0] + 1
@@ -14,7 +16,7 @@ SELECT CASE src_thrift.lint[0]
         WHEN '0' THEN src_thrift.lstring
         ELSE NULL
        END)[0]
-FROM src_thrift LIMIT 3;
+FROM src_thrift tablesample (3 rows);
 
 SELECT CASE src_thrift.lint[0]
         WHEN 0 THEN src_thrift.lint[0] + 1
@@ -31,4 +33,4 @@ SELECT CASE src_thrift.lint[0]
         WHEN '0' THEN src_thrift.lstring
         ELSE NULL
        END)[0]
-FROM src_thrift LIMIT 3;
+FROM src_thrift tablesample (3 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q
index 48ca29cbc3ba647d2fda7245cc23f2677b24d905..d3c417babd4662d16b24060c10b8c4362887c651 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION coalesce;
 DESCRIBE FUNCTION EXTENDED coalesce;
 
@@ -20,7 +22,7 @@ SELECT COALESCE(1),
        COALESCE(NULL, 2.0, 3.0),
        COALESCE(2.0, NULL, 3.0),
        COALESCE(IF(TRUE, NULL, 0), NULL)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT COALESCE(1),
        COALESCE(1, 2),
@@ -40,7 +42,7 @@ SELECT COALESCE(1),
        COALESCE(NULL, 2.0, 3.0),
        COALESCE(2.0, NULL, 3.0),
        COALESCE(IF(TRUE, NULL, 0), NULL)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 EXPLAIN
 SELECT COALESCE(src_thrift.lint[1], 999),
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q
index 6c12f81304e46705faa1fa45ca93bc529ce4b4fb..c7983b8eb20500155dccc6708acded0741d504db 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 EXPLAIN
 CREATE TEMPORARY FUNCTION test_udf_get_java_string AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestGetJavaString';
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q
index f642f6a2d00df1859c9610c17f6ee6e901405204..e35a1cfa170ca507b4c24de3bd64820c4ebab5a4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION concat;
 DESCRIBE FUNCTION EXTENDED concat;
 
@@ -12,4 +14,10 @@ SELECT
   concat(1, 2),
   concat(1),
   concat('1234', 'abc', 'extra argument')
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
+
+-- binary/mixed
+SELECT
+  concat(cast('ab' as binary), cast('cd' as binary)),
+  concat('ab', cast('cd' as binary))
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q
index 6a0ce20dc37b76c2a5308b7cdb231b7da3784979..538dfae06f20da93844c166f3a4c057a918db963 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION concat_ws;
 DESCRIBE FUNCTION EXTENDED concat_ws;
 
@@ -24,7 +26,7 @@ SELECT concat_ws('.', array('www', 'face', 'book', 'com'), '1234'),
        concat_ws('_', array('www', 'face'), array('book', 'com', '1234')),
        concat_ws('**', 'www', array('face'), array('book', 'com', '1234')),
        concat_ws('[]', array('www'), 'face', array('book', 'com', '1234')),
-       concat_ws('AAA', array('www'), array('face', 'book', 'com'), '1234') FROM dest1 LIMIT 1;
+       concat_ws('AAA', array('www'), array('face', 'book', 'com'), '1234') FROM dest1 tablesample (1 rows);
 
 SELECT concat_ws('.', array('www', 'face', 'book', 'com'), '1234'),
        concat_ws('-', 'www', array('face', 'book', 'com'), '1234'),
@@ -32,7 +34,7 @@ SELECT concat_ws('.', array('www', 'face', 'book', 'com'), '1234'),
        concat_ws('_', array('www', 'face'), array('book', 'com', '1234')),
        concat_ws('**', 'www', array('face'), array('book', 'com', '1234')),
        concat_ws('[]', array('www'), 'face', array('book', 'com', '1234')),
-       concat_ws('AAA', array('www'), array('face', 'book', 'com'), '1234') FROM dest1 LIMIT 1;
+       concat_ws('AAA', array('www'), array('face', 'book', 'com'), '1234') FROM dest1 tablesample (1 rows);
 
 SELECT concat_ws(NULL, array('www', 'face', 'book', 'com'), '1234'),
        concat_ws(NULL, 'www', array('face', 'book', 'com'), '1234'),
@@ -40,4 +42,4 @@ SELECT concat_ws(NULL, array('www', 'face', 'book', 'com'), '1234'),
        concat_ws(NULL, array('www', 'face'), array('book', 'com', '1234')),
        concat_ws(NULL, 'www', array('face'), array('book', 'com', '1234')),
        concat_ws(NULL, array('www'), 'face', array('book', 'com', '1234')),
-       concat_ws(NULL, array('www'), array('face', 'book', 'com'), '1234') FROM dest1 LIMIT 1;
+       concat_ws(NULL, array('www'), array('face', 'book', 'com'), '1234') FROM dest1 tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q
index 212bcfb57938f24c97b6f22dc712144e34172cc6..c6d6cf8600c8cb6f5a27dc1e3c26245622c04ec3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION conv;
 DESCRIBE FUNCTION EXTENDED conv;
 
@@ -9,7 +11,7 @@ SELECT
   conv('22', 10, 10),
   conv('110011', 2, 16),
   conv('facebook', 36, 16)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 -- Test negative numbers. If to_base is positive, the number should be handled
 -- as a two's complement (64-bit)
@@ -18,7 +20,7 @@ SELECT
   conv('1011', 2, -16),
   conv('-1', 10, 16),
   conv('-15', 10, 16)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 -- Test overflow. If a number is two large, the result should be -1 (if signed)
 -- or MAX_LONG (if unsigned)
@@ -27,7 +29,7 @@ SELECT
   conv('9223372036854775807', 36, -16),
   conv('-9223372036854775807', 36, 16),
   conv('-9223372036854775807', 36, -16)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 -- Test with invalid input. If one of the bases is invalid, the result should
 -- be NULL. If there is an invalid digit in the number, the longest valid
@@ -37,7 +39,7 @@ SELECT
   conv('131', 1, 5),
   conv('515', 5, 100),
   conv('10', -2, 2)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 -- Perform the same tests with number arguments.
 
@@ -45,31 +47,31 @@ SELECT
   conv(4521, 10, 36),
   conv(22, 10, 10),
   conv(110011, 2, 16)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   conv(-641, 10, -10),
   conv(1011, 2, -16),
   conv(-1, 10, 16),
   conv(-15, 10, 16)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   conv(9223372036854775807, 36, 16),
   conv(9223372036854775807, 36, -16),
   conv(-9223372036854775807, 36, 16),
   conv(-9223372036854775807, 36, -16)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   conv(123455, 3, 10),
   conv(131, 1, 5),
   conv(515, 5, 100),
   conv('10', -2, 2)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 -- Make sure that state is properly reset.
 
 SELECT conv(key, 10, 16),
        conv(key, 16, 10)
-FROM src LIMIT 3;
+FROM src tablesample (3 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q
index 7887c4c2603f19041ab975be81481a9a7226bb81..11ef8d7d87b284723a16d08e75071eae5de1de20 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q
@@ -1,8 +1,10 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION cos;
 DESCRIBE FUNCTION EXTENDED cos;
 
 SELECT cos(null)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT cos(0.98), cos(1.57), cos(-0.5)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_current_database.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_current_database.q
new file mode 100644
index 0000000000000000000000000000000000000000..4ada035d877263bc6a939f8d19473bdf71fc0a41
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_current_database.q
@@ -0,0 +1,26 @@
+DESCRIBE FUNCTION current_database;
+
+explain
+select current_database();
+select current_database();
+
+create database xxx;
+use xxx;
+
+explain
+select current_database();
+select current_database();
+
+set hive.fetch.task.conversion=more;
+
+use default;
+
+explain
+select current_database();
+select current_database();
+
+use xxx;
+
+explain
+select current_database();
+select current_database();
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q
index 014ca1c6e6afb939525214997e36ed3111cf512b..d5360fe3b22f7936f60a3df9d46281965f89a9da 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q
@@ -1,14 +1,16 @@
-explain 
-select degrees(PI()) FROM src LIMIT 1;
+set hive.fetch.task.conversion=more;
+
+explain
+select degrees(PI()) FROM src tablesample (1 rows);
 
-select degrees(PI()) FROM src LIMIT 1;
+select degrees(PI()) FROM src tablesample (1 rows);
 
 DESCRIBE FUNCTION degrees;
 DESCRIBE FUNCTION EXTENDED degrees;
 explain 
-select degrees(PI()) FROM src LIMIT 1;
+select degrees(PI()) FROM src tablesample (1 rows);
 
-select degrees(PI()) FROM src LIMIT 1;
+select degrees(PI()) FROM src tablesample (1 rows);
 
 DESCRIBE FUNCTION degrees;
 DESCRIBE FUNCTION EXTENDED degrees;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q
index 4229e625a042af0f938395e0ff0404c2dd31c529..b0d2844d423467cb0cd41e9f5ebeda27c5877826 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q
@@ -1,4 +1,6 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION div;
 DESCRIBE FUNCTION EXTENDED div;
 
-SELECT 3 DIV 2 FROM SRC LIMIT 1;
+SELECT 3 DIV 2 FROM SRC tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q
index dc4b2e7884d68616fc5e889b1529446b241fac90..d36ba99118f511c4e4cb44a82753c0cfebe9a10c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q
@@ -1,4 +1,6 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION /;
 DESCRIBE FUNCTION EXTENDED /;
 
-SELECT 3 / 2 FROM SRC LIMIT 1;
+SELECT 3 / 2 FROM SRC tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q
index c32340ac89feb0a4c8771b2a5db211eded27b5c5..fae764965a9a68342885f2501e8c53b99c95f365 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION elt;
 DESCRIBE FUNCTION EXTENDED elt;
 
@@ -13,7 +15,7 @@ SELECT elt(2, 'abc', 'defg'),
        elt(null, 'abc', 'defg'),
        elt(0, 'abc', 'defg'),
        elt(3, 'abc', 'defg')
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT elt(2, 'abc', 'defg'),
        elt(3, 'aa', 'bb', 'cc', 'dd', 'ee', 'ff', 'gg'),
@@ -26,4 +28,4 @@ SELECT elt(2, 'abc', 'defg'),
        elt(null, 'abc', 'defg'),
        elt(0, 'abc', 'defg'),
        elt(3, 'abc', 'defg')
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q
index f5e9a7df4be00553541dd7c85116e86604310026..ea9b18bf1fad207d61be2d804a6256b412b12f66 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q
@@ -1,12 +1,14 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION =;
 DESCRIBE FUNCTION EXTENDED =;
 
 DESCRIBE FUNCTION ==;
 DESCRIBE FUNCTION EXTENDED ==;
 
-SELECT true=false, false=true, false=false, true=true, NULL=NULL, true=NULL, NULL=true, false=NULL, NULL=false FROM src LIMIT 1;
+SELECT true=false, false=true, false=false, true=true, NULL=NULL, true=NULL, NULL=true, false=NULL, NULL=false FROM src tablesample (1 rows);
 
 DESCRIBE FUNCTION <=>;
 DESCRIBE FUNCTION EXTENDED <=>;
 
-SELECT true<=>false, false<=>true, false<=>false, true<=>true, NULL<=>NULL, true<=>NULL, NULL<=>true, false<=>NULL, NULL<=>false FROM src LIMIT 1;
+SELECT true<=>false, false<=>true, false<=>false, true<=>true, NULL<=>NULL, true<=>NULL, NULL<=>true, false<=>NULL, NULL<=>false FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q
index 19af288ff840454c8588f0524081f5f2e4fcaaab..ae651644a778a4eca3aee19241042900c8846e3e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q
@@ -1,22 +1,24 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION explode;
 DESCRIBE FUNCTION EXTENDED explode;
 
-EXPLAIN EXTENDED SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3;
-EXPLAIN EXTENDED SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3) a GROUP BY a.myCol;
+EXPLAIN EXTENDED SELECT explode(array(1,2,3)) AS myCol FROM src tablesample (1 rows);
+EXPLAIN EXTENDED SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src tablesample (1 rows)) a GROUP BY a.myCol;
 
-SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3;
-SELECT explode(array(1,2,3)) AS (myCol) FROM src LIMIT 3;
-SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3) a GROUP BY a.myCol;
+SELECT explode(array(1,2,3)) AS myCol FROM src tablesample (1 rows);
+SELECT explode(array(1,2,3)) AS (myCol) FROM src tablesample (1 rows);
+SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src tablesample (1 rows)) a GROUP BY a.myCol;
 
-EXPLAIN EXTENDED SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src LIMIT 3;
-EXPLAIN EXTENDED SELECT a.key, a.val, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src LIMIT 3) a GROUP BY a.key, a.val;
+EXPLAIN EXTENDED SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src tablesample (1 rows);
+EXPLAIN EXTENDED SELECT a.key, a.val, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src tablesample (1 rows)) a GROUP BY a.key, a.val;
 
-SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src LIMIT 3;
-SELECT a.key, a.val, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src LIMIT 3) a GROUP BY a.key, a.val;
+SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src tablesample (1 rows);
+SELECT a.key, a.val, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src tablesample (1 rows)) a GROUP BY a.key, a.val;
 
 drop table lazy_array_map;
 create table lazy_array_map (map_col map<int,string>, array_col array<string>);
-INSERT OVERWRITE TABLE lazy_array_map select map(1,'one',2,'two',3,'three'), array('100','200','300') FROM src LIMIT 1;
+INSERT OVERWRITE TABLE lazy_array_map select map(1,'one',2,'two',3,'three'), array('100','200','300') FROM src tablesample (1 rows);
 
 SELECT array_col, myCol from lazy_array_map lateral view explode(array_col) X AS myCol;
 SELECT map_col, myKey, myValue from lazy_array_map lateral view explode(map_col) X AS myKey, myValue;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q
index e995f5cf3c084b8cad308f0b8b1f2c10c505ab9b..be92c024d3ed236bf9d74d6b728b9fa8b6b52f7b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION field;
 DESCRIBE FUNCTION EXTENDED field;
 
@@ -5,7 +7,7 @@ SELECT
   field("x", "a", "b", "c", "d"),
   field(NULL, "a", "b", "c", "d"),
   field(0, 1, 2, 3, 4)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   field("a", "a", "b", "c", "d"),
@@ -13,7 +15,7 @@ SELECT
   field("c", "a", "b", "c", "d"),
   field("d", "a", "b", "c", "d"),
   field("d", "a", "b", NULL, "d")
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   field(1, 1, 2, 3, 4),
@@ -21,11 +23,11 @@ SELECT
   field(3, 1, 2, 3, 4),
   field(4, 1, 2, 3, 4),
   field(4, 1, 2, NULL, 4)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 
 CREATE TABLE test_table(col1 STRING, col2 STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE test_table;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE test_table;
 
 select col1,col2,
   field("66",col1),
@@ -42,7 +44,7 @@ from test_table where col1="86" or col1="66";
 
 
 CREATE TABLE test_table1(col1 int, col2 string) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE test_table1;
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE test_table1;
 
 select col1,col2,
   field(66,col1),
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q
index eac2c6e91c3d148cfcd1122d886e4a8c232ae63b..72c65b419808e4cf5279a645435fa026c0991376 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION find_in_set;
 DESCRIBE FUNCTION EXTENDED find_in_set; 
 
@@ -6,18 +8,18 @@ FROM src1 SELECT find_in_set(src1.key,concat(src1.key,',',src1.value));
 
 FROM src1 SELECT find_in_set(src1.key,concat(src1.key,',',src1.value));
 
-SELECT find_in_set('ab','ab,abc,abcde') FROM src1 LIMIT 1;
-SELECT find_in_set('ab','abc,ab,bbb') FROM src1 LIMIT 1;
-SELECT find_in_set('ab','def,abc,ab') FROM src1 LIMIT 1;
-SELECT find_in_set('ab','abc,abd,abf') FROM src1 LIMIT 1;
-SELECT find_in_set(null,'a,b,c') FROM src1 LIMIT 1;
-SELECT find_in_set('a',null) FROM src1 LIMIT 1;
-SELECT find_in_set('', '') FROM src1 LIMIT 1;
-SELECT find_in_set('',',') FROM src1 LIMIT 1;
-SELECT find_in_set('','a,,b') FROM src1 LIMIT 1;
-SELECT find_in_set('','a,b,') FROM src1 LIMIT 1;
-SELECT find_in_set(',','a,b,d,') FROM src1 LIMIT 1;
-SELECT find_in_set('a','') FROM src1 LIMIT 1;
-SELECT find_in_set('a,','a,b,c,d') FROM src1 LIMIT 1;
+SELECT find_in_set('ab','ab,abc,abcde') FROM src1 tablesample (1 rows);
+SELECT find_in_set('ab','abc,ab,bbb') FROM src1 tablesample (1 rows);
+SELECT find_in_set('ab','def,abc,ab') FROM src1 tablesample (1 rows);
+SELECT find_in_set('ab','abc,abd,abf') FROM src1 tablesample (1 rows);
+SELECT find_in_set(null,'a,b,c') FROM src1 tablesample (1 rows);
+SELECT find_in_set('a',null) FROM src1 tablesample (1 rows);
+SELECT find_in_set('', '') FROM src1 tablesample (1 rows);
+SELECT find_in_set('',',') FROM src1 tablesample (1 rows);
+SELECT find_in_set('','a,,b') FROM src1 tablesample (1 rows);
+SELECT find_in_set('','a,b,') FROM src1 tablesample (1 rows);
+SELECT find_in_set(',','a,b,d,') FROM src1 tablesample (1 rows);
+SELECT find_in_set('a','') FROM src1 tablesample (1 rows);
+SELECT find_in_set('a,','a,b,c,d') FROM src1 tablesample (1 rows);
 
 SELECT * FROM src1 WHERE NOT find_in_set(key,'311,128,345,2,956')=0;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q
index e2084cddf0ec7e300966981e1c6bd0714f19e517..2504bd0b68306156c9b302dc72f8a4ecd99541b9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 use default;
 -- Test format_number() UDF
 
@@ -7,26 +9,26 @@ DESCRIBE FUNCTION EXTENDED format_number;
 EXPLAIN
 SELECT format_number(12332.123456, 4),
     format_number(12332.1,4),
-    format_number(12332.2,0) FROM src limit 1;
+    format_number(12332.2,0) FROM src tablesample (1 rows);
 
 SELECT format_number(12332.123456, 4),
     format_number(12332.1,4),
     format_number(12332.2,0)
-FROM src limit 1;
+FROM src tablesample (1 rows);
 
 -- positive numbers
 SELECT format_number(0.123456789, 12),
     format_number(12345678.123456789, 5),
     format_number(1234567.123456789, 7),
     format_number(123456.123456789, 0)
-FROM src limit 1;
+FROM src tablesample (1 rows);
 
 -- negative numbers
 SELECT format_number(-123456.123456789, 0),
     format_number(-1234567.123456789, 2),
     format_number(-0.123456789, 15),
     format_number(-12345.123456789, 4)
-FROM src limit 1;
+FROM src tablesample (1 rows);
 
 -- zeros
 SELECT format_number(0.0, 4),
@@ -34,7 +36,7 @@ SELECT format_number(0.0, 4),
     format_number(000.0000, 1),
     format_number(00000.0000, 1),
     format_number(-00.0, 4)
-FROM src limit 1;
+FROM src tablesample (1 rows);
 
 -- integers
 SELECT format_number(0, 0),
@@ -42,7 +44,7 @@ SELECT format_number(0, 0),
     format_number(12, 2),
     format_number(123, 5),
     format_number(1234, 7)
-FROM src limit 1;
+FROM src tablesample (1 rows);
 
 -- long and double boundary
 -- 9223372036854775807 is LONG_MAX
@@ -54,4 +56,4 @@ SELECT format_number(-9223372036854775807, 10),
     format_number(9223372036854775807, 20),
     format_number(4.9E-324, 324),
     format_number(1.7976931348623157E308, 308)
-FROM src limit 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q
index 464f2df3dcd7a5485d89d7d6ada458dd55c9d07a..05f7f5a9811bc028565d88f08fc36058cd85148d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION get_json_object;
 DESCRIBE FUNCTION EXTENDED get_json_object;
 
@@ -5,6 +7,8 @@ CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE;
 
 FROM src INSERT OVERWRITE TABLE dest1 SELECT '  abc  ' WHERE src.key = 86;
 
+set hive.fetch.task.conversion=more;
+
 EXPLAIN
 SELECT get_json_object(src_json.json, '$.owner') FROM src_json;
 
@@ -33,8 +37,8 @@ SELECT get_json_object(src_json.json, '$.fb:testid') FROM src_json;
 
 CREATE TABLE dest2(c1 STRING) STORED AS RCFILE;
 
-INSERT OVERWRITE TABLE dest2 SELECT '{"a":"b\nc"}' FROM src LIMIT 1;
+INSERT OVERWRITE TABLE dest2 SELECT '{"a":"b\nc"}' FROM src tablesample (1 rows);
 
 SELECT * FROM dest2;
 
-SELECT get_json_object(c1, '$.a') FROM dest2;
\ No newline at end of file
+SELECT get_json_object(c1, '$.a') FROM dest2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q
index aea110a9431cf591f06e1f6700a1c91142dbdf00..230bd244ffbc7f319474eb2afcb3fece8bd89143 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q
@@ -1,4 +1,6 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION >;
 DESCRIBE FUNCTION EXTENDED >;
 
-SELECT true>false, false>true, false>false, true>true FROM src LIMIT 1;
\ No newline at end of file
+SELECT true>false, false>true, false>false, true>true FROM src tablesample (1 rows);
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q
index 8de165b9ea9ed3d38fa4f2511abb29e0938395d5..025eed7dd558222d268309ff133c63ea162fa1df 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q
@@ -1,4 +1,6 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION >=;
 DESCRIBE FUNCTION EXTENDED >=;
 
-SELECT true>=false, false>=true, false>=false, true>=true FROM src LIMIT 1;
\ No newline at end of file
+SELECT true>=false, false>=true, false>=false, true>=true FROM src tablesample (1 rows);
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q
index faf372218a10767b48bfe2e093cf41b6988fd56c..5814a1779bd16dbd274e6e7f97a41fc0fadaa22d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION hash;
 DESCRIBE FUNCTION EXTENDED hash;
 
@@ -7,11 +9,11 @@ SELECT hash(CAST(1 AS TINYINT)), hash(CAST(2 AS SMALLINT)),
        hash(CAST(1.25 AS FLOAT)), hash(CAST(16.0 AS DOUBLE)),
        hash('400'), hash('abc'), hash(TRUE), hash(FALSE),
        hash(1, 2, 3)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT hash(CAST(1 AS TINYINT)), hash(CAST(2 AS SMALLINT)),
        hash(3), hash(CAST('123456789012' AS BIGINT)),
        hash(CAST(1.25 AS FLOAT)), hash(CAST(16.0 AS DOUBLE)),
        hash('400'), hash('abc'), hash(TRUE), hash(FALSE),
        hash(1, 2, 3)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q
index 37e035ad42b00cac06e0efb007e2582ca9f779d0..0e5457965a71e6c31315ac547bf5b19dab2477f3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION hex;
 DESCRIBE FUNCTION EXTENDED hex;
 
@@ -7,14 +9,14 @@ SELECT
   hex('Facebook'),
   hex('\0'),
   hex('qwertyuiopasdfghjkl')
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 -- If the argument is a number, hex should convert it to hexadecimal.
 SELECT
   hex(1),
   hex(0),
   hex(4207849477)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 -- Negative numbers should be treated as two's complement (64 bit).
-SELECT hex(-5) FROM src LIMIT 1;
+SELECT hex(-5) FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q
index c5c366daa8c839c7f3c182f7a2502029ed7521f2..b9811e6c6f6521f208892139f6917a926241e49e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION hour;
 DESCRIBE FUNCTION EXTENDED hour;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q
index 4f7c8b4a36bad8f4a756a6f857f10da590fd7977..d9285ff7ce19d0a7f9ffc482549261c9f357205c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION if;
 DESCRIBE FUNCTION EXTENDED if;
 
@@ -8,7 +10,7 @@ SELECT IF(TRUE, 1, 2) AS COL1,
        IF(2=2, 1, NULL) AS COL4,
        IF(2=2, NULL, 1) AS COL5,
        IF(IF(TRUE, NULL, FALSE), 1, 2) AS COL6
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 
 SELECT IF(TRUE, 1, 2) AS COL1,
@@ -17,7 +19,7 @@ SELECT IF(TRUE, 1, 2) AS COL1,
        IF(2=2, 1, NULL) AS COL4,
        IF(2=2, NULL, 1) AS COL5,
        IF(IF(TRUE, NULL, FALSE), 1, 2) AS COL6
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 -- Type conversions
 EXPLAIN
@@ -25,10 +27,10 @@ SELECT IF(TRUE, CAST(128 AS SMALLINT), CAST(1 AS TINYINT)) AS COL1,
        IF(FALSE, 1, 1.1) AS COL2,
        IF(FALSE, 1, 'ABC') AS COL3,
        IF(FALSE, 'ABC', 12.3) AS COL4
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT IF(TRUE, CAST(128 AS SMALLINT), CAST(1 AS TINYINT)) AS COL1,
        IF(FALSE, 1, 1.1) AS COL2,
        IF(FALSE, 1, 'ABC') AS COL3,
        IF(FALSE, 'ABC', 12.3) AS COL4
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q
index 75778138742d5179553a80251aa3530f2e354598..a7ce3c6f0bdbb1c42ae8855aec9d59a64531ccb6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 SELECT 1 IN (1, 2, 3),
        4 IN (1, 2, 3),
        array(1,2,3) IN (array(1,2,3)),
@@ -8,6 +10,6 @@ SELECT 1 IN (1, 2, 3),
        1 IN (1, 2, 3) OR false IN(false),
        NULL IN (1, 2, 3),
        4 IN (1, 2, 3, NULL),
-       (1+3) IN (5, 6, (1+2) + 1) FROM src LIMIT 1;
+       (1+3) IN (5, 6, (1+2) + 1) FROM src tablesample (1 rows);
 
 SELECT key FROM src WHERE key IN ("238", 86);
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q
index 4da478908ddbdd2cf4b7b1bdca053ee7985411df..9d9efe8e23d6e73429ee5cd2c8470359ba2b3498 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q
@@ -1,12 +1,12 @@
 DESCRIBE FUNCTION in_file;
 
 EXPLAIN
-SELECT in_file("303", "../data/files/test2.dat"),
-       in_file("304", "../data/files/test2.dat"),
-       in_file(CAST(NULL AS STRING), "../data/files/test2.dat")
+SELECT in_file("303", "../../data/files/test2.dat"),
+       in_file("304", "../../data/files/test2.dat"),
+       in_file(CAST(NULL AS STRING), "../../data/files/test2.dat")
 FROM src LIMIT 1;
 
-SELECT in_file("303", "../data/files/test2.dat"),
-       in_file("304", "../data/files/test2.dat"),
-       in_file(CAST(NULL AS STRING), "../data/files/test2.dat")
+SELECT in_file("303", "../../data/files/test2.dat"),
+       in_file("304", "../../data/files/test2.dat"),
+       in_file(CAST(NULL AS STRING), "../../data/files/test2.dat")
 FROM src LIMIT 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q
index 9079d0e5743f65848af5a8713a13ba25b8e2562a..6844f9d0bfc6ce812d3368c8fd2cf198ca8408b6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q
@@ -1,2 +1,3 @@
+set hive.support.quoted.identifiers=none;
 DESCRIBE FUNCTION `index`;
 DESCRIBE FUNCTION EXTENDED `index`;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q
index 39006f4b9b11774bf54c9717af114dc9f9a44250..95d55f71c11cd94956752836f0b959bcc855df61 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 describe function inline;
 
 explain SELECT inline( 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q
index 20ed8e4ea01761bad3f90020a29589a231d7e6b3..790a1049d177712df3085906ef1fd9d46e349904 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION instr;
 DESCRIBE FUNCTION EXTENDED instr;
 
@@ -15,7 +17,7 @@ SELECT instr('abcd', 'abc'),
        instr(CAST(16.0 AS DOUBLE), '.0'),
        instr(null, 'abc'),
        instr('abcd', null)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT instr('abcd', 'abc'),
        instr('abcabc', 'ccc'),
@@ -30,4 +32,4 @@ SELECT instr('abcd', 'abc'),
        instr(CAST(16.0 AS DOUBLE), '.0'),
        instr(null, 'abc'),
        instr('abcd', null)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q
index d1569cc7f6dbb4f0e3ae1bf08fddc0d554062a4f..efb834efdc64f16a46d4121e800c45a6591872d5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION isnull;
 DESCRIBE FUNCTION EXTENDED isnull;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q
index 2f28be1e9d0e1458c78f265a9477f6109a0ac258..51280b2567ccad2b7cb65c328f5687df81d0203f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION java_method;
 DESCRIBE FUNCTION EXTENDED java_method;
 
@@ -11,7 +13,7 @@ SELECT java_method("java.lang.String", "valueOf", 1),
        java_method("java.lang.Math", "round", 2.5),
        java_method("java.lang.Math", "exp", 1.0),
        java_method("java.lang.Math", "floor", 1.9)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 
 SELECT java_method("java.lang.String", "valueOf", 1),
@@ -21,5 +23,5 @@ SELECT java_method("java.lang.String", "valueOf", 1),
        java_method("java.lang.Math", "round", 2.5),
        java_method("java.lang.Math", "exp", 1.0),
        java_method("java.lang.Math", "floor", 1.9)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q
index b84307970d6306655c7f4150050b49c66edee1ee..4413751ae647b87c44751eab22362efaa4435bdc 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION length;
 DESCRIBE FUNCTION EXTENDED length;
 
@@ -9,6 +11,6 @@ DROP TABLE dest1;
 
 -- Test with non-ascii characters. 
 CREATE TABLE dest1(name STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv4.txt' INTO TABLE dest1;
+LOAD DATA LOCAL INPATH '../../data/files/kv4.txt' INTO TABLE dest1;
 EXPLAIN SELECT length(dest1.name) FROM dest1;
 SELECT length(dest1.name) FROM dest1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q
index a2577006a885d3417131e1a01fe59e5ad84f8001..03326777bfa9fddf1a991279b635d6933bf5e655 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q
@@ -1,4 +1,6 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION <;
 DESCRIBE FUNCTION EXTENDED <;
 
-SELECT true<false, false<true, false<false, true<true FROM src LIMIT 1;
+SELECT true<false, false<true, false<false, true<true FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthanorequal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthanorequal.q
index e741b12e7230547f2f353a17ea20dfa5b80a3e55..ae7109fb92b28903137e4f9fec631b4fa37c55d7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthanorequal.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthanorequal.q
@@ -1,4 +1,6 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION <=;
 DESCRIBE FUNCTION EXTENDED <=;
 
-SELECT true<=false, false<=true, false<=false, true<=true FROM src LIMIT 1;
+SELECT true<=false, false<=true, false<=false, true<=true FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_like.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_like.q
index 02c2924fca35bdddda3de101364ad6b2332b5600..12983bdceb69b639c69ed5a4c0e71450173cddde 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_like.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_like.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION like;
 DESCRIBE FUNCTION EXTENDED like;
 
@@ -18,4 +20,4 @@ SELECT '1+2' LIKE '_+_',
        '112' LIKE '1+_',
        '|||' LIKE '|_|', 
        '+++' LIKE '1+_' 
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_locate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_locate.q
index 80148d09233066a41ff747bfdc66f12d246f573f..68216d597da577f25d42a761cd28f83f6ea96a13 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_locate.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_locate.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION locate;
 DESCRIBE FUNCTION EXTENDED locate;
 
@@ -19,7 +21,7 @@ SELECT locate('abc', 'abcd'),
        locate('abc', null),
        locate('abc', 'abcd', null),
        locate('abc', 'abcd', 'invalid number')
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT locate('abc', 'abcd'),
        locate('ccc', 'abcabc'),
@@ -38,4 +40,4 @@ SELECT locate('abc', 'abcd'),
        locate('abc', null),
        locate('abc', 'abcd', null),
        locate('abc', 'abcd', 'invalid number')
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_logic_java_boolean.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_logic_java_boolean.q
index 508f9fd5582e14df66204661a99d83a8a34cc573..a4aa6bc8e861ce7e12c95b953feaa8743f642f63 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_logic_java_boolean.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_logic_java_boolean.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 EXPLAIN
 CREATE TEMPORARY FUNCTION test_udf_get_java_boolean AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestGetJavaBoolean';
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lpad.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lpad.q
index 8879231aeeb95ad4bd882daf1e0e541d8b43bd54..937c92a29ece3c08b7c7120b1a66f2cac0cf5cb1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lpad.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lpad.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION lpad;
 DESCRIBE FUNCTION EXTENDED lpad;
 
@@ -5,10 +7,10 @@ EXPLAIN SELECT
   lpad('hi', 1, '?'),
   lpad('hi', 5, '.'),
   lpad('hi', 6, '123')
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   lpad('hi', 1, '?'),
   lpad('hi', 5, '.'),
   lpad('hi', 6, '123')
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map.q
index e975131bd55996471e18420e64680cab91c769bd..e1923b9ae097345a13d899134c806bd8977afc1d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map.q
@@ -1,8 +1,10 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION map;
 DESCRIBE FUNCTION EXTENDED map;
 
 EXPLAIN SELECT map(), map(1, "a", 2, "b", 3, "c"), map(1, 2, "a", "b"), 
-map(1, "a", 2, "b", 3, "c")[2],  map(1, 2, "a", "b")["a"], map(1, array("a"))[1][0] FROM src LIMIT 1;
+map(1, "a", 2, "b", 3, "c")[2],  map(1, 2, "a", "b")["a"], map(1, array("a"))[1][0] FROM src tablesample (1 rows);
 
 SELECT map(), map(1, "a", 2, "b", 3, "c"), map(1, 2, "a", "b"), 
-map(1, "a", 2, "b", 3, "c")[2],  map(1, 2, "a", "b")["a"], map(1, array("a"))[1][0] FROM src LIMIT 1;
+map(1, "a", 2, "b", 3, "c")[2],  map(1, 2, "a", "b")["a"], map(1, array("a"))[1][0] FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map_keys.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map_keys.q
index 7ae8d78823ce9f9386d09d4c0141c1a364a4fb0a..78778e73cd83442f9f4fcfd2cda0dc0561ac8fce 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map_keys.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map_keys.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 use default;
 -- Test map_keys() UDF
 
@@ -5,7 +7,7 @@ DESCRIBE FUNCTION map_keys;
 DESCRIBE FUNCTION EXTENDED map_keys;
 
 -- Evaluate function against INT valued keys
-SELECT map_keys(map(1, "a", 2, "b", 3, "c")) FROM src LIMIT 1;
+SELECT map_keys(map(1, "a", 2, "b", 3, "c")) FROM src tablesample (1 rows);
 
 -- Evaluate function against STRING valued keys
-SELECT map_keys(map("a", 1, "b", 2, "c", 3)) FROM src LIMIT 1;
+SELECT map_keys(map("a", 1, "b", 2, "c", 3)) FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map_values.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map_values.q
index e25b9bc6697599cd073c3a387f5b21c137687021..4b55873fb971484ec05619326b8d9e803613b37d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map_values.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map_values.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 use default;
 -- Test map_values() UDF
 
@@ -5,7 +7,7 @@ DESCRIBE FUNCTION map_values;
 DESCRIBE FUNCTION EXTENDED map_values;
 
 -- Evaluate function against STRING valued values
-SELECT map_values(map(1, "a", 2, "b", 3, "c")) FROM src LIMIT 1;
+SELECT map_values(map(1, "a", 2, "b", 3, "c")) FROM src tablesample (1 rows);
 
 -- Evaluate function against INT valued keys
-SELECT map_values(map("a", 1, "b", 2, "c", 3)) FROM src LIMIT 1;
+SELECT map_values(map("a", 1, "b", 2, "c", 3)) FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_named_struct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_named_struct.q
index bbf0f67d8140803604eb8eb06fead818c12e5101..ad6fd7a021fc76988ac7b3800805e11323bbbc6a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_named_struct.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_named_struct.q
@@ -1,9 +1,11 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION named_struct;
 DESCRIBE FUNCTION EXTENDED named_struct;
 
 EXPLAIN
 SELECT named_struct("foo", 1, "bar", 2),
-       named_struct("foo", 1, "bar", 2).foo FROM src LIMIT 1;
+       named_struct("foo", 1, "bar", 2).foo FROM src tablesample (1 rows);
 
 SELECT named_struct("foo", 1, "bar", 2),
-       named_struct("foo", 1, "bar", 2).foo FROM src LIMIT 1;
+       named_struct("foo", 1, "bar", 2).foo FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_negative.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_negative.q
index 6c069382140771c95c563b4690e58d4109791564..b038c8cad57f295689aa70ef9782b60b92e6a093 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_negative.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_negative.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION negative;
 DESCRIBE FUNCTION EXTENDED negative;
 
@@ -5,9 +7,9 @@ DESCRIBE FUNCTION EXTENDED negative;
 DESCRIBE FUNCTION -;
 DESCRIBE FUNCTION EXTENDED -;
 
-select - null from src limit 1;
-select - cast(null as int) from src limit 1;
-select - cast(null as smallint) from src limit 1;
-select - cast(null as bigint) from src limit 1;
-select - cast(null as double) from src limit 1;
-select - cast(null as float) from src limit 1;
+select - null from src tablesample (1 rows);
+select - cast(null as int) from src tablesample (1 rows);
+select - cast(null as smallint) from src tablesample (1 rows);
+select - cast(null as bigint) from src tablesample (1 rows);
+select - cast(null as double) from src tablesample (1 rows);
+select - cast(null as float) from src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notequal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notequal.q
index e1a15098db369c8168e3402f9a461480c53afc5f..138110f6edd64bcf7d827aff43f75cc56610f8c8 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notequal.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notequal.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION <>;
 DESCRIBE FUNCTION EXTENDED <>;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q
index 88396545de933f9734d35824760268f64d04746f..dceab7edaa1fad39f60e9d60c7ec6ae4d6b059ca 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 SELECT 1 NOT IN (1, 2, 3),
        4 NOT IN (1, 2, 3),
        1 = 2 NOT IN (true, false),
@@ -7,4 +9,4 @@ SELECT 1 NOT IN (1, 2, 3),
        "abc" NOT RLIKE "^bc",
        "abc" NOT REGEXP "^ab",
        "abc" NOT REGEXP "^bc",
-       1 IN (1, 2) AND "abc" NOT LIKE "bc%" FROM src LIMIT 1;
+       1 IN (1, 2) AND "abc" NOT LIKE "bc%" FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q
index 0133b4b90bcbc45af1b29c981c9b9754f72a0359..97162576df167ccf8a5a362f85317125e81a0a25 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q
@@ -1,3 +1,4 @@
+set hive.fetch.task.conversion=more;
 
 DESCRIBE FUNCTION nvl;
 DESCRIBE FUNCTION EXTENDED nvl;
@@ -5,9 +6,9 @@ DESCRIBE FUNCTION EXTENDED nvl;
 EXPLAIN
 SELECT NVL( 1 , 2 ) AS COL1,
        NVL( NULL, 5 ) AS COL2
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT NVL( 1 , 2 ) AS COL1,
        NVL( NULL, 5 ) AS COL2
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q
index 9ff73d42b41e6f6abf387df33319146b552b8216..d42a2f337ebca61df31d13f00d9ef02776902b68 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q
@@ -1,20 +1,22 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION pmod;
 DESCRIBE FUNCTION EXTENDED pmod;
 
 SELECT pmod(null, null)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT pmod(-100,9), pmod(-50,101), pmod(-1000,29)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT pmod(100,19), pmod(50,125), pmod(300,15)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
-SELECT pmod(CAST(-100 AS TINYINT),CAST(9 AS TINYINT)), pmod(CAST(-50 AS TINYINT),CAST(101 AS TINYINT)), pmod(CAST(-100 AS TINYINT),CAST(29 AS TINYINT)) FROM src LIMIT 1;
-SELECT pmod(CAST(-100 AS SMALLINT),CAST(9 AS SMALLINT)), pmod(CAST(-50 AS SMALLINT),CAST(101 AS SMALLINT)), pmod(CAST(-100 AS SMALLINT),CAST(29 AS SMALLINT)) FROM src LIMIT 1;
-SELECT pmod(CAST(-100 AS BIGINT),CAST(9 AS BIGINT)), pmod(CAST(-50 AS BIGINT),CAST(101 AS BIGINT)), pmod(CAST(-100 AS BIGINT),CAST(29 AS BIGINT)) FROM src LIMIT 1;
+SELECT pmod(CAST(-100 AS TINYINT),CAST(9 AS TINYINT)), pmod(CAST(-50 AS TINYINT),CAST(101 AS TINYINT)), pmod(CAST(-100 AS TINYINT),CAST(29 AS TINYINT)) FROM src tablesample (1 rows);
+SELECT pmod(CAST(-100 AS SMALLINT),CAST(9 AS SMALLINT)), pmod(CAST(-50 AS SMALLINT),CAST(101 AS SMALLINT)), pmod(CAST(-100 AS SMALLINT),CAST(29 AS SMALLINT)) FROM src tablesample (1 rows);
+SELECT pmod(CAST(-100 AS BIGINT),CAST(9 AS BIGINT)), pmod(CAST(-50 AS BIGINT),CAST(101 AS BIGINT)), pmod(CAST(-100 AS BIGINT),CAST(29 AS BIGINT)) FROM src tablesample (1 rows);
 
-SELECT pmod(CAST(-100.91 AS FLOAT),CAST(9.8 AS FLOAT)), pmod(CAST(-50.1 AS FLOAT),CAST(101.8 AS FLOAT)), pmod(CAST(-100.91 AS FLOAT),CAST(29.75 AS FLOAT)) FROM src LIMIT 1;
-SELECT pmod(CAST(-100.91 AS DOUBLE),CAST(9.8 AS DOUBLE)), pmod(CAST(-50.1 AS DOUBLE),CAST(101.8 AS DOUBLE)), pmod(CAST(-100.91 AS DOUBLE),CAST(29.75 AS DOUBLE)) FROM src LIMIT 1;
-SELECT pmod(CAST(-100.91 AS DECIMAL),CAST(9.8 AS DECIMAL)), pmod(CAST(-50.1 AS DECIMAL),CAST(101.8 AS DECIMAL)), pmod(CAST(-100.91 AS DECIMAL),CAST(29.75 AS DECIMAL)) FROM src LIMIT 1;
+SELECT pmod(CAST(-100.91 AS FLOAT),CAST(9.8 AS FLOAT)), pmod(CAST(-50.1 AS FLOAT),CAST(101.8 AS FLOAT)), pmod(CAST(-100.91 AS FLOAT),CAST(29.75 AS FLOAT)) FROM src tablesample (1 rows);
+SELECT pmod(CAST(-100.91 AS DOUBLE),CAST(9.8 AS DOUBLE)), pmod(CAST(-50.1 AS DOUBLE),CAST(101.8 AS DOUBLE)), pmod(CAST(-100.91 AS DOUBLE),CAST(29.75 AS DOUBLE)) FROM src tablesample (1 rows);
+SELECT pmod(CAST(-100.91 AS DECIMAL(5,2)),CAST(9.8 AS DECIMAL(2,1))), pmod(CAST(-50.1 AS DECIMAL(3,1)),CAST(101.8 AS DECIMAL(4,1))), pmod(CAST(-100.91 AS DECIMAL(5,2)),CAST(29.75 AS DECIMAL(4,2))) FROM src tablesample (1 rows);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q
index 99e89ccbf773a6ab4b8e595287cc7c5b8d5f91da..115e4e56f1806f342c2587e0bbbef1a3d53fef78 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q
@@ -4,21 +4,23 @@ use default;
 DESCRIBE FUNCTION printf;
 DESCRIBE FUNCTION EXTENDED printf;
 
+set hive.fetch.task.conversion=more;
+
 EXPLAIN
-SELECT printf("Hello World %d %s", 100, "days") FROM src LIMIT 1;
+SELECT printf("Hello World %d %s", 100, "days") FROM src tablesample (1 rows);
 
 -- Test Primitive Types
-SELECT printf("Hello World %d %s", 100, "days") FROM src LIMIT 1;
-SELECT printf("All Type Test: %b, %c, %d, %e, %+10.4f, %g, %h, %s, %a", false, 65, 15000, 12.3400, 27183.240051, 2300.41, 50, "corret", 256.125) FROM src LIMIT 1;
+SELECT printf("Hello World %d %s", 100, "days") FROM src tablesample (1 rows);
+SELECT printf("All Type Test: %b, %c, %d, %e, %+10.4f, %g, %h, %s, %a", false, 65, 15000, 12.3400, 27183.240051, 2300.41, 50, "corret", 256.125) FROM src tablesample (1 rows);
 
 -- Test NULL Values
-SELECT printf("Color %s, String Null: %s, number1 %d, number2 %05d, Integer Null: %d, hex %#x, float %5.2f Double Null: %f\n", "red", NULL, 123456, 89, NULL, 255, 3.14159, NULL) FROM src LIMIT 1;
+SELECT printf("Color %s, String Null: %s, number1 %d, number2 %05d, Integer Null: %d, hex %#x, float %5.2f Double Null: %f\n", "red", NULL, 123456, 89, NULL, 255, 3.14159, NULL) FROM src tablesample (1 rows);
 
 -- Test Timestamp
 create table timestamp_udf (t timestamp);
-from src
+from (select * from src tablesample (1 rows)) s
   insert overwrite table timestamp_udf
-    select '2011-05-06 07:08:09.1234567' limit 1;
+    select '2011-05-06 07:08:09.1234567';
 select printf("timestamp: %s", t) from timestamp_udf;
 drop table timestamp_udf;
 
@@ -27,7 +29,7 @@ CREATE TABLE binay_udf(key binary, value int)
 ROW FORMAT DELIMITED
 FIELDS TERMINATED BY '9'
 STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/string.txt' INTO TABLE binay_udf;
+LOAD DATA LOCAL INPATH '../../data/files/string.txt' INTO TABLE binay_udf;
 create table dest1 (key binary, value int);
 insert overwrite table dest1 select transform(*) using 'cat' as key binary, value int from binay_udf;
 select value, printf("format key: %s", key) from dest1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q
index 001d1cf8b646466518aa10add32520d02838ac8f..19242bd757b416c52087e06d839ed77ebc39a57e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q
@@ -1,16 +1,18 @@
-explain 
-select radians(57.2958) FROM src LIMIT 1;
+set hive.fetch.task.conversion=more;
+
+explain
+select radians(57.2958) FROM src tablesample (1 rows);
 
-select radians(57.2958) FROM src LIMIT 1;
-select radians(143.2394) FROM src LIMIT 1;
+select radians(57.2958) FROM src tablesample (1 rows);
+select radians(143.2394) FROM src tablesample (1 rows);
 
 DESCRIBE FUNCTION radians;
 DESCRIBE FUNCTION EXTENDED radians;
 explain 
-select radians(57.2958) FROM src LIMIT 1;
+select radians(57.2958) FROM src tablesample (1 rows);
 
-select radians(57.2958) FROM src LIMIT 1;
-select radians(143.2394) FROM src LIMIT 1;
+select radians(57.2958) FROM src tablesample (1 rows);
+select radians(143.2394) FROM src tablesample (1 rows);
 
 DESCRIBE FUNCTION radians;
 DESCRIBE FUNCTION EXTENDED radians;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q
index f357ff50495641e22f254bca67c0423c9f7bcfa5..cef1e4a5d9195d754fa8600f3293106838d15bf9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION reflect;
 DESCRIBE FUNCTION EXTENDED reflect;
 
@@ -10,7 +12,7 @@ SELECT reflect("java.lang.String", "valueOf", 1),
        reflect("java.lang.Math", "exp", 1.0),
        reflect("java.lang.Math", "floor", 1.9),
        reflect("java.lang.Integer", "valueOf", key, 16)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 
 SELECT reflect("java.lang.String", "valueOf", 1),
@@ -21,4 +23,4 @@ SELECT reflect("java.lang.String", "valueOf", 1),
        reflect("java.lang.Math", "exp", 1.0),
        reflect("java.lang.Math", "floor", 1.9),
        reflect("java.lang.Integer", "valueOf", key, 16)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q
index 9ffd755292f665df7174f9bd1a4f23c4ec7f0755..a65294b3358444b8c63d3b8933eb6563cc30dc77 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION reflect2;
 DESCRIBE FUNCTION EXTENDED reflect2;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q
index 3aee10963d74796f2c596bb66ce8493fb66c0999..12b685b32c69c83dd9b41f0947a59abfc5152a89 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q
@@ -1,6 +1,8 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION regexp;
 DESCRIBE FUNCTION EXTENDED regexp;
 
 SELECT 'fofo' REGEXP '^fo', 'fo\no' REGEXP '^fo\no$', 'Bn' REGEXP '^Ba*n', 'afofo' REGEXP 'fo',
 'afofo' REGEXP '^fo', 'Baan' REGEXP '^Ba?n', 'axe' REGEXP 'pi|apa', 'pip' REGEXP '^(pi)*$'
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q
index 162085f4c71c8f528baef6c7db371a7472000e62..91474bac2a16b87f2d57af877799e45a88a6e447 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION repeat;
 DESCRIBE FUNCTION EXTENDED repeat;
 
@@ -6,11 +8,11 @@ EXPLAIN SELECT
   repeat("", 4),
   repeat("asd", 0),
   repeat("asdf", -1)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   repeat("Facebook", 3),
   repeat("", 4),
   repeat("asd", 0),
   repeat("asdf", -1)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q
index 81f765ec5937cb91092195344d0f937499485ff8..89aafe3443eebc083d8cf098fcba871957be9805 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q
@@ -11,5 +11,5 @@ DROP TABLE dest1;
 -- kv4.txt contains the text 0xE982B5E993AE, which should be reversed to
 -- 0xE993AEE982B5
 CREATE TABLE dest1(name STRING) STORED AS TEXTFILE;
-LOAD DATA LOCAL INPATH '../data/files/kv4.txt' INTO TABLE dest1;
+LOAD DATA LOCAL INPATH '../../data/files/kv4.txt' INTO TABLE dest1;
 SELECT count(1) FROM dest1 WHERE reverse(dest1.name) = _UTF-8 0xE993AEE982B5;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q
index 18ebba870849307a5c088a683352b763f9a1f068..88b22749a3095379d77b657b2a97d97ba3b0af39 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q
@@ -1,15 +1,17 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION round;
 DESCRIBE FUNCTION EXTENDED round;
 
 SELECT round(null), round(null, 0), round(125, null), 
 round(1.0/0.0, 0), round(power(-1.0,0.5), 0)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   round(55555), round(55555, 0), round(55555, 1), round(55555, 2), round(55555, 3),
   round(55555, -1), round(55555, -2), round(55555, -3), round(55555, -4),
   round(55555, -5), round(55555, -6), round(55555, -7), round(55555, -8)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   round(125.315), round(125.315, 0),
@@ -18,7 +20,7 @@ SELECT
   round(-125.315), round(-125.315, 0),
   round(-125.315, 1), round(-125.315, 2), round(-125.315, 3), round(-125.315, 4),
   round(-125.315, -1), round(-125.315, -2), round(-125.315, -3), round(-125.315, -4)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   round(3.141592653589793, -15), round(3.141592653589793, -16),
@@ -38,7 +40,7 @@ SELECT
   round(3.141592653589793, 12), round(3.141592653589793, 13),
   round(3.141592653589793, 13), round(3.141592653589793, 14),
   round(3.141592653589793, 15), round(3.141592653589793, 16)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
-SELECT round(1809242.3151111344, 9), round(-1809242.3151111344, 9)
-FROM src LIMIT 1;
+SELECT round(1809242.3151111344, 9), round(-1809242.3151111344, 9), round(1809242.3151111344BD, 9), round(-1809242.3151111344BD, 9)
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q
index 6be30855aea1bc7e9ec37c8efc5ef8de870c5ca1..43988c1225cd8a0fbbdc52cbc0a922a3e44d3bc9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q
@@ -1,8 +1,10 @@
--- test for NaN (not-a-number) 
+set hive.fetch.task.conversion=more;
+
+-- test for NaN (not-a-number)
 create table tstTbl1(n double);
 
 insert overwrite table tstTbl1
-select 'NaN' from src limit 1;
+select 'NaN' from src tablesample (1 rows);
 
 select * from tstTbl1;
 
@@ -10,4 +12,4 @@ select round(n, 1) from tstTbl1;
 select round(n) from tstTbl1;
 
 -- test for Infinity
-select round(1/0), round(1/0, 2), round(1.0/0.0), round(1.0/0.0, 2) from src limit 1;
+select round(1/0), round(1/0, 2), round(1.0/0.0), round(1.0/0.0, 2) from src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q
index 50a1f44419047ae83610c4f9b3b485b48d5f03a4..f042b6f3fa9c7935c621d34f93623bc2d313983b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q
@@ -1,14 +1,16 @@
+set hive.fetch.task.conversion=more;
+
 -- test for TINYINT
-select round(-128), round(127), round(0) from src limit 1;
+select round(-128), round(127), round(0) from src tablesample (1 rows);
 
 -- test for SMALLINT
-select round(-32768), round(32767), round(-129), round(128) from src limit 1;
+select round(-32768), round(32767), round(-129), round(128) from src tablesample (1 rows);
 
 -- test for INT
-select round(cast(negative(pow(2, 31)) as INT)), round(cast((pow(2, 31) - 1) as INT)), round(-32769), round(32768) from src limit 1;
+select round(cast(negative(pow(2, 31)) as INT)), round(cast((pow(2, 31) - 1) as INT)), round(-32769), round(32768) from src tablesample (1 rows);
 
 -- test for BIGINT
-select round(cast(negative(pow(2, 63)) as BIGINT)), round(cast((pow(2, 63) - 1) as BIGINT)), round(cast(negative(pow(2, 31) + 1) as BIGINT)), round(cast(pow(2, 31) as BIGINT)) from src limit 1;
+select round(cast(negative(pow(2, 63)) as BIGINT)), round(cast((pow(2, 63) - 1) as BIGINT)), round(cast(negative(pow(2, 31) + 1) as BIGINT)), round(cast(pow(2, 31) as BIGINT)) from src tablesample (1 rows);
 
 -- test for DOUBLE
-select round(126.1), round(126.7), round(32766.1), round(32766.7) from src limit 1;
+select round(126.1), round(126.7), round(32766.1), round(32766.7) from src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q
index 01e5fbd429b1744dcce63498fdda82977cb0a8c1..4ee69e89850753beb717c1361c2ce71db8bb48fe 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION rpad;
 DESCRIBE FUNCTION EXTENDED rpad;
 
@@ -5,10 +7,10 @@ EXPLAIN SELECT
   rpad('hi', 1, '?'),
   rpad('hi', 5, '.'),
   rpad('hi', 6, '123')
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   rpad('hi', 1, '?'),
   rpad('hi', 5, '.'),
   rpad('hi', 6, '123')
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q
index 1943188086ea55cf54c1f9da17097e5ce24b973f..f63426d7e7257cc4dc43ea7b15a3d33f45775be2 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION second;
 DESCRIBE FUNCTION EXTENDED second;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q
index abceb3439ec1e72d2595a060b9a3267d7ce1f59e..b1602e8a257efbeb05f98a852ab99925934f815d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q
@@ -1,20 +1,22 @@
-explain 
-select sign(0) FROM src LIMIT 1;
-select sign(0) FROM src LIMIT 1;
+set hive.fetch.task.conversion=more;
+
+explain
+select sign(0) FROM src tablesample (1 rows);
+select sign(0) FROM src tablesample (1 rows);
 
-select sign(-45) FROM src LIMIT 1;
+select sign(-45) FROM src tablesample (1 rows);
 
-select sign(46)  FROM src LIMIT 1;
+select sign(46)  FROM src tablesample (1 rows);
 
 DESCRIBE FUNCTION sign;
 DESCRIBE FUNCTION EXTENDED sign;
 explain 
-select sign(0) FROM src LIMIT 1;
-select sign(0) FROM src LIMIT 1;
+select sign(0) FROM src tablesample (1 rows);
+select sign(0) FROM src tablesample (1 rows);
 
-select sign(-45) FROM src LIMIT 1;
+select sign(-45) FROM src tablesample (1 rows);
 
-select sign(46)  FROM src LIMIT 1;
+select sign(46)  FROM src tablesample (1 rows);
 
 DESCRIBE FUNCTION sign;
 DESCRIBE FUNCTION EXTENDED sign;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q
index abb7cac8dac9d7a0cb9af086c48f3d6d3432bdac..79745be772f6f961530215df177f56f8eb8ef84a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q
@@ -1,8 +1,10 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION sin;
 DESCRIBE FUNCTION EXTENDED sin;
 
 SELECT sin(null)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT sin(0.98), sin(1.57), sin(-0.5)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q
index 8aaa68a6e318a8c7c58947e5141d5523e572dcca..f6f76a30e72587ad22b5f6b6f2cff311a859bbb1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION size;
 DESCRIBE FUNCTION EXTENDED size;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q
index ef0973212a27c50aa05f475f5c1cd10f6a5d7099..313bcf8a1ea52e84032517ae43d555f8265244c6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 use default;
 -- Test sort_array() UDF
 
@@ -6,16 +8,16 @@ DESCRIBE FUNCTION EXTENDED sort_array;
 
 -- Evaluate function against STRING valued keys
 EXPLAIN
-SELECT sort_array(array("b", "d", "c", "a")) FROM src LIMIT 1;
+SELECT sort_array(array("b", "d", "c", "a")) FROM src tablesample (1 rows);
 
-SELECT sort_array(array("f", "a", "g", "c", "b", "d", "e")) FROM src LIMIT 1;
-SELECT sort_array(sort_array(array("hadoop distributed file system", "enterprise databases", "hadoop map-reduce"))) FROM src LIMIT 1;
+SELECT sort_array(array("f", "a", "g", "c", "b", "d", "e")) FROM src tablesample (1 rows);
+SELECT sort_array(sort_array(array("hadoop distributed file system", "enterprise databases", "hadoop map-reduce"))) FROM src tablesample (1 rows);
 
 -- Evaluate function against INT valued keys
-SELECT sort_array(array(2, 9, 7, 3, 5, 4, 1, 6, 8)) FROM src LIMIT 1;
+SELECT sort_array(array(2, 9, 7, 3, 5, 4, 1, 6, 8)) FROM src tablesample (1 rows);
 
 -- Evaluate function against FLOAT valued keys
-SELECT sort_array(sort_array(array(2.333, 9, 1.325, 2.003, 0.777, -3.445, 1))) FROM src LIMIT 1;
+SELECT sort_array(sort_array(array(2.333, 9, 1.325, 2.003, 0.777, -3.445, 1))) FROM src tablesample (1 rows);
 
 -- Test it against data in a table.
 CREATE TABLE dest1 (
@@ -30,7 +32,7 @@ CREATE TABLE dest1 (
 	timestamps ARRAY<TIMESTAMP>
 ) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/primitive_type_arrays.txt' OVERWRITE INTO TABLE dest1;
+LOAD DATA LOCAL INPATH '../../data/files/primitive_type_arrays.txt' OVERWRITE INTO TABLE dest1;
 
 SELECT	sort_array(tinyints), sort_array(smallints), sort_array(ints),
 	sort_array(bigints), sort_array(booleans), sort_array(floats),
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q
index cf6466fb63bad60f10316b1abc85c5dd384cf3fd..cc616f784fcea70fe3fb1ff33f144cd92adcfd19 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION space;
 DESCRIBE FUNCTION EXTENDED space;
 
@@ -7,7 +9,7 @@ EXPLAIN SELECT
   space(1),
   space(-1),
   space(-100)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   length(space(10)),
@@ -15,7 +17,7 @@ SELECT
   length(space(1)),
   length(space(-1)),
   length(space(-100))
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   space(10),
@@ -23,5 +25,5 @@ SELECT
   space(1),
   space(-1),
   space(-100)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q
index f79901736cf7c5089fc6198bc96d41de575e99c2..55919eac743b700440880848df15610737640d3a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION split;
 DESCRIBE FUNCTION EXTENDED split;
 
@@ -6,11 +8,11 @@ EXPLAIN SELECT
   split('oneAtwoBthreeC', '[ABC]'),
   split('', '.'),
   split(50401020, 0)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT 
   split('a b c', ' '),
   split('oneAtwoBthreeC', '[ABC]'),
   split('', '.'),
   split(50401020, 0)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q
index 3ee19c83699bb6413b9a290d1bc27908d0d36527..ee2135b509ae2d5751b482222989ad36a03a3c0d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q
@@ -1,9 +1,11 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION struct;
 DESCRIBE FUNCTION EXTENDED struct;
 
 EXPLAIN
 SELECT struct(1), struct(1, "a"), struct(1, "b", 1.5).col1, struct(1, struct("a", 1.5)).col2.col1
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT struct(1), struct(1, "a"), struct(1, "b", 1.5).col1, struct(1, struct("a", 1.5)).col2.col1
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q
index 32757bef51c1c77cb7f0b4d178d8e6268fc32807..2d04f904bbc6ff3ce7dd92477fb84d884a2c8c87 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION substr;
 DESCRIBE FUNCTION EXTENDED substr;
 
@@ -5,7 +7,7 @@ SELECT
   substr(null, 1), substr(null, 1, 1),
   substr('ABC', null), substr('ABC', null, 1),
   substr('ABC', 1, null)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   substr('ABC', 1, 0), substr('ABC', 1, -1), substr('ABC', 2, -100),
@@ -14,7 +16,7 @@ SELECT
   substr('ABC', 100), substr('ABC', 100, 100),
   substr('ABC', -100), substr('ABC', -100, 100),
   substr('ABC', 2147483647), substr('ABC', 2147483647, 2147483647)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   substr('ABCDEFG', 3, 4), substr('ABCDEFG', -5, 4),
@@ -22,7 +24,7 @@ SELECT
   substr('ABC', 0), substr('ABC', 1), substr('ABC', 2), substr('ABC', 3),
   substr('ABC', 1, 2147483647), substr('ABC', 2, 2147483647),
   substr('A', 0), substr('A', 1), substr('A', -1)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   substr('ABC', 0, 1), substr('ABC', 0, 2), substr('ABC', 0, 3), substr('ABC', 0, 4),
@@ -30,14 +32,14 @@ SELECT
   substr('ABC', 2, 1), substr('ABC', 2, 2), substr('ABC', 2, 3), substr('ABC', 2, 4),
   substr('ABC', 3, 1), substr('ABC', 3, 2), substr('ABC', 3, 3), substr('ABC', 3, 4),
   substr('ABC', 4, 1)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT
   substr('ABC', -1, 1), substr('ABC', -1, 2), substr('ABC', -1, 3), substr('ABC', -1, 4),
   substr('ABC', -2, 1), substr('ABC', -2, 2), substr('ABC', -2, 3), substr('ABC', -2, 4),
   substr('ABC', -3, 1), substr('ABC', -3, 2), substr('ABC', -3, 3), substr('ABC', -3, 4),
   substr('ABC', -4, 1)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 -- substring() is a synonim of substr(), so just perform some basic tests
 SELECT
@@ -46,7 +48,7 @@ SELECT
   substring('ABC', 0), substring('ABC', 1), substring('ABC', 2), substring('ABC', 3),
   substring('ABC', 1, 2147483647), substring('ABC', 2, 2147483647),
   substring('A', 0), substring('A', 1), substring('A', -1)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 -- test for binary substr
 SELECT
@@ -63,7 +65,7 @@ SELECT
   substr(ABC, -3, 1), substr(ABC, -3, 2), substr(ABC, -3, 3), substr(ABC, -3, 4),
   substr(ABC, -4, 1)
 FROM (
-   select CAST(concat(substr(value, 1, 0), 'ABC') as BINARY) as ABC from src LIMIT 1
+   select CAST(concat(substr(value, 1, 0), 'ABC') as BINARY) as ABC from src tablesample (1 rows)
 ) X;
 
 -- test UTF-8 substr
@@ -72,4 +74,4 @@ SELECT
   substr("abc 玩", 5),
   substr("abc 玩玩玩 abc", 5),
   substr("abc 玩玩玩 abc", 5, 3)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q
index f103da9ecb2de08db1aa3a0da8d80c0089b045ec..3980fe83fbcb244f7b9c10ecf2d82842ef14a046 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q
@@ -1,16 +1,18 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION tan;
 DESCRIBE FUNCTION EXTENDED tan;
 
 SELECT tan(null)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT tan(1), tan(6), tan(-1.0)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 DESCRIBE FUNCTION tan;
 DESCRIBE FUNCTION EXTENDED tan;
 
 SELECT tan(null)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT tan(1), tan(6), tan(-1.0)
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q
index 322a061d645d7997e1e222362847c30daaf5ac4c..c94a52133d32437082d0de158821fd22f96a7898 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q
@@ -1,12 +1,10 @@
+set hive.fetch.task.conversion=more;
+
 EXPLAIN
 CREATE TEMPORARY FUNCTION testlength AS 'org.apache.hadoop.hive.ql.udf.UDFTestLength';
 
 CREATE TEMPORARY FUNCTION testlength AS 'org.apache.hadoop.hive.ql.udf.UDFTestLength';
 
-CREATE TABLE dest1(len INT);
-
-FROM src INSERT OVERWRITE TABLE dest1 SELECT testlength(src.value);
-
-SELECT dest1.* FROM dest1;
+SELECT testlength(src.value) FROM src;
 
 DROP TEMPORARY FUNCTION testlength;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q
index 6de270902f7fb0c3c0c29b62244f0e90e41af39a..27e46c24a83d50ed17178e8fe0dfa3987e8966ff 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q
@@ -1,12 +1,10 @@
+set hive.fetch.task.conversion=more;
+
 EXPLAIN
 CREATE TEMPORARY FUNCTION testlength2 AS 'org.apache.hadoop.hive.ql.udf.UDFTestLength2';
 
 CREATE TEMPORARY FUNCTION testlength2 AS 'org.apache.hadoop.hive.ql.udf.UDFTestLength2';
 
-CREATE TABLE dest1(len INT);
-
-FROM src INSERT OVERWRITE TABLE dest1 SELECT testlength2(src.value);
-
-SELECT dest1.* FROM dest1;
+SELECT testlength2(src.value) FROM src;
 
 DROP TEMPORARY FUNCTION testlength2;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q
index ca23f719f93e30ff8c603734571b20d26f13b8e1..8bea7abcbc4d3b9c30d7deb6245cc9f55072875b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q
@@ -1,44 +1,46 @@
--- 'true' cases: 
+set hive.fetch.task.conversion=more;
 
-SELECT CAST(CAST(1 AS TINYINT) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(2 AS SMALLINT) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(-4 AS INT) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(-444 AS BIGINT) AS BOOLEAN) FROM src LIMIT 1;
+-- 'true' cases:
 
-SELECT CAST(CAST(7.0 AS FLOAT) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(-8.0 AS DOUBLE) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(-99.0 AS DECIMAL) AS BOOLEAN) FROM src LIMIT 1;
+SELECT CAST(CAST(1 AS TINYINT) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(2 AS SMALLINT) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-4 AS INT) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-444 AS BIGINT) AS BOOLEAN) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST('Foo' AS STRING) AS BOOLEAN) FROM src LIMIT 1;
+SELECT CAST(CAST(7.0 AS FLOAT) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-8.0 AS DOUBLE) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-99.0 AS DECIMAL) AS BOOLEAN) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST('2011-05-06 07:08:09' as timestamp) AS BOOLEAN) FROM src LIMIT 1;
+SELECT CAST(CAST('Foo' AS STRING) AS BOOLEAN) FROM src tablesample (1 rows);
+
+SELECT CAST(CAST('2011-05-06 07:08:09' as timestamp) AS BOOLEAN) FROM src tablesample (1 rows);
 
 -- 'false' cases: 
 
-SELECT CAST(CAST(0 AS TINYINT) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(0 AS SMALLINT) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(0 AS INT) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(0 AS BIGINT) AS BOOLEAN) FROM src LIMIT 1;
+SELECT CAST(CAST(0 AS TINYINT) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(0 AS SMALLINT) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(0 AS INT) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(0 AS BIGINT) AS BOOLEAN) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(0.0 AS FLOAT) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(0.0 AS DOUBLE) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(0.0 AS DECIMAL) AS BOOLEAN) FROM src LIMIT 1;
+SELECT CAST(CAST(0.0 AS FLOAT) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(0.0 AS DOUBLE) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(0.0 AS DECIMAL) AS BOOLEAN) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST('' AS STRING) AS BOOLEAN) FROM src LIMIT 1;
+SELECT CAST(CAST('' AS STRING) AS BOOLEAN) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(0 as timestamp) AS BOOLEAN) FROM src LIMIT 1;
+SELECT CAST(CAST(0 as timestamp) AS BOOLEAN) FROM src tablesample (1 rows);
 
 -- 'NULL' cases:
-SELECT CAST(NULL AS BOOLEAN) FROM src LIMIT 1;
+SELECT CAST(NULL AS BOOLEAN) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(NULL AS TINYINT) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(NULL AS SMALLINT) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(NULL AS INT) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(NULL AS BIGINT) AS BOOLEAN) FROM src LIMIT 1;
+SELECT CAST(CAST(NULL AS TINYINT) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(NULL AS SMALLINT) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(NULL AS INT) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(NULL AS BIGINT) AS BOOLEAN) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(NULL AS FLOAT) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(NULL AS DOUBLE) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(NULL AS DECIMAL) AS BOOLEAN) FROM src LIMIT 1;
+SELECT CAST(CAST(NULL AS FLOAT) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(NULL AS DOUBLE) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(NULL AS DECIMAL) AS BOOLEAN) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(NULL AS STRING) AS BOOLEAN) FROM src LIMIT 1;
-SELECT CAST(CAST(NULL as timestamp) AS BOOLEAN) FROM src LIMIT 1;
+SELECT CAST(CAST(NULL AS STRING) AS BOOLEAN) FROM src tablesample (1 rows);
+SELECT CAST(CAST(NULL as timestamp) AS BOOLEAN) FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q
index ded930d1115a4db466529455fa88de17bb1cfac6..aa0a250e4678de1bf224d1f2ec9a5eaaead149e8 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q
@@ -1,15 +1,17 @@
--- Conversion of main primitive types to Byte type: 
-SELECT CAST(NULL AS TINYINT) FROM src LIMIT 1;
+set hive.fetch.task.conversion=more;
 
-SELECT CAST(TRUE AS TINYINT) FROM src LIMIT 1;
+-- Conversion of main primitive types to Byte type:
+SELECT CAST(NULL AS TINYINT) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(-18 AS SMALLINT) AS TINYINT) FROM src LIMIT 1;
-SELECT CAST(-129 AS TINYINT) FROM src LIMIT 1;
-SELECT CAST(CAST(-1025 AS BIGINT) AS TINYINT) FROM src LIMIT 1;
+SELECT CAST(TRUE AS TINYINT) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(-3.14 AS DOUBLE) AS TINYINT) FROM src LIMIT 1;
-SELECT CAST(CAST(-3.14 AS FLOAT) AS TINYINT) FROM src LIMIT 1;
-SELECT CAST(CAST(-3.14 AS DECIMAL) AS TINYINT) FROM src LIMIT 1;
+SELECT CAST(CAST(-18 AS SMALLINT) AS TINYINT) FROM src tablesample (1 rows);
+SELECT CAST(-129 AS TINYINT) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-1025 AS BIGINT) AS TINYINT) FROM src tablesample (1 rows);
 
-SELECT CAST('-38' AS TINYINT) FROM src LIMIT 1;
+SELECT CAST(CAST(-3.14 AS DOUBLE) AS TINYINT) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-3.14 AS FLOAT) AS TINYINT) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-3.14 AS DECIMAL) AS TINYINT) FROM src tablesample (1 rows);
+
+SELECT CAST('-38' AS TINYINT) FROM src tablesample (1 rows);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q
index b0a248ad70bbac04adb2df27898309d377d9b30a..005ec9d24e1ecd45c4e84461da0210b00badded5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q
@@ -1,15 +1,17 @@
--- Conversion of main primitive types to Double type: 
-SELECT CAST(NULL AS DOUBLE) FROM src LIMIT 1;
+set hive.fetch.task.conversion=more;
 
-SELECT CAST(TRUE AS DOUBLE) FROM src LIMIT 1;
+-- Conversion of main primitive types to Double type:
+SELECT CAST(NULL AS DOUBLE) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(-7 AS TINYINT) AS DOUBLE) FROM src LIMIT 1;
-SELECT CAST(CAST(-18 AS SMALLINT) AS DOUBLE) FROM src LIMIT 1;
-SELECT CAST(-129 AS DOUBLE) FROM src LIMIT 1;
-SELECT CAST(CAST(-1025 AS BIGINT) AS DOUBLE) FROM src LIMIT 1;
+SELECT CAST(TRUE AS DOUBLE) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(-3.14 AS FLOAT) AS DOUBLE) FROM src LIMIT 1;
-SELECT CAST(CAST(-3.14 AS DECIMAL) AS DOUBLE) FROM src LIMIT 1;
+SELECT CAST(CAST(-7 AS TINYINT) AS DOUBLE) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-18 AS SMALLINT) AS DOUBLE) FROM src tablesample (1 rows);
+SELECT CAST(-129 AS DOUBLE) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-1025 AS BIGINT) AS DOUBLE) FROM src tablesample (1 rows);
 
-SELECT CAST('-38.14' AS DOUBLE) FROM src LIMIT 1;
+SELECT CAST(CAST(-3.14 AS FLOAT) AS DOUBLE) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-3.14 AS DECIMAL(3,2)) AS DOUBLE) FROM src tablesample (1 rows);
+
+SELECT CAST('-38.14' AS DOUBLE) FROM src tablesample (1 rows);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q
index c91d18cc2f5c740398651130f19632be944fbd94..95671f15fe0ec7bce015003625ae4e8c9808e30a 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q
@@ -1,15 +1,17 @@
--- Conversion of main primitive types to Float type: 
-SELECT CAST(NULL AS FLOAT) FROM src LIMIT 1;
+set hive.fetch.task.conversion=more;
 
-SELECT CAST(TRUE AS FLOAT) FROM src LIMIT 1;
+-- Conversion of main primitive types to Float type:
+SELECT CAST(NULL AS FLOAT) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(-7 AS TINYINT) AS FLOAT) FROM src LIMIT 1;
-SELECT CAST(CAST(-18 AS SMALLINT) AS FLOAT) FROM src LIMIT 1;
-SELECT CAST(-129 AS FLOAT) FROM src LIMIT 1;
-SELECT CAST(CAST(-1025 AS BIGINT) AS FLOAT) FROM src LIMIT 1;
+SELECT CAST(TRUE AS FLOAT) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(-3.14 AS DOUBLE) AS FLOAT) FROM src LIMIT 1;
-SELECT CAST(CAST(-3.14 AS DECIMAL) AS FLOAT) FROM src LIMIT 1;
+SELECT CAST(CAST(-7 AS TINYINT) AS FLOAT) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-18 AS SMALLINT) AS FLOAT) FROM src tablesample (1 rows);
+SELECT CAST(-129 AS FLOAT) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-1025 AS BIGINT) AS FLOAT) FROM src tablesample (1 rows);
 
-SELECT CAST('-38.14' AS FLOAT) FROM src LIMIT 1;
+SELECT CAST(CAST(-3.14 AS DOUBLE) AS FLOAT) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-3.14 AS DECIMAL(3,2)) AS FLOAT) FROM src tablesample (1 rows);
+
+SELECT CAST('-38.14' AS FLOAT) FROM src tablesample (1 rows);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q
index 45dc6f8bd32fad360ed23be5ae1d5ee2b06fc4e6..706411a398463b0967cd21a63b46445f8fa895e6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q
@@ -1,15 +1,17 @@
--- Conversion of main primitive types to Long type: 
-SELECT CAST(NULL AS BIGINT) FROM src LIMIT 1;
+set hive.fetch.task.conversion=more;
 
-SELECT CAST(TRUE AS BIGINT) FROM src LIMIT 1;
+-- Conversion of main primitive types to Long type:
+SELECT CAST(NULL AS BIGINT) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(-7 AS TINYINT) AS BIGINT) FROM src LIMIT 1;
-SELECT CAST(CAST(-18 AS SMALLINT) AS BIGINT) FROM src LIMIT 1;
-SELECT CAST(-129 AS BIGINT) FROM src LIMIT 1;
+SELECT CAST(TRUE AS BIGINT) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(-3.14 AS DOUBLE) AS BIGINT) FROM src LIMIT 1;
-SELECT CAST(CAST(-3.14 AS FLOAT) AS BIGINT) FROM src LIMIT 1;
-SELECT CAST(CAST(-3.14 AS DECIMAL) AS BIGINT) FROM src LIMIT 1;
+SELECT CAST(CAST(-7 AS TINYINT) AS BIGINT) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-18 AS SMALLINT) AS BIGINT) FROM src tablesample (1 rows);
+SELECT CAST(-129 AS BIGINT) FROM src tablesample (1 rows);
 
-SELECT CAST('-38' AS BIGINT) FROM src LIMIT 1;
+SELECT CAST(CAST(-3.14 AS DOUBLE) AS BIGINT) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-3.14 AS FLOAT) AS BIGINT) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-3.14 AS DECIMAL) AS BIGINT) FROM src tablesample (1 rows);
+
+SELECT CAST('-38' AS BIGINT) FROM src tablesample (1 rows);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q
index 7d843c1ea12ea339c6723032971353e4cb13a884..5cc4e57c8c9b598f3092663fe62334850c11e6d7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q
@@ -1,15 +1,17 @@
--- Conversion of main primitive types to Short type: 
-SELECT CAST(NULL AS SMALLINT) FROM src LIMIT 1;
+set hive.fetch.task.conversion=more;
 
-SELECT CAST(TRUE AS SMALLINT) FROM src LIMIT 1;
+-- Conversion of main primitive types to Short type:
+SELECT CAST(NULL AS SMALLINT) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(-18 AS TINYINT) AS SMALLINT) FROM src LIMIT 1;
-SELECT CAST(-129 AS SMALLINT) FROM src LIMIT 1;
-SELECT CAST(CAST(-1025 AS BIGINT) AS SMALLINT) FROM src LIMIT 1;
+SELECT CAST(TRUE AS SMALLINT) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(-3.14 AS DOUBLE) AS SMALLINT) FROM src LIMIT 1;
-SELECT CAST(CAST(-3.14 AS FLOAT) AS SMALLINT) FROM src LIMIT 1;
-SELECT CAST(CAST(-3.14 AS DECIMAL) AS SMALLINT) FROM src LIMIT 1;
+SELECT CAST(CAST(-18 AS TINYINT) AS SMALLINT) FROM src tablesample (1 rows);
+SELECT CAST(-129 AS SMALLINT) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-1025 AS BIGINT) AS SMALLINT) FROM src tablesample (1 rows);
 
-SELECT CAST('-38' AS SMALLINT) FROM src LIMIT 1;
+SELECT CAST(CAST(-3.14 AS DOUBLE) AS SMALLINT) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-3.14 AS FLOAT) AS SMALLINT) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-3.14 AS DECIMAL) AS SMALLINT) FROM src tablesample (1 rows);
+
+SELECT CAST('-38' AS SMALLINT) FROM src tablesample (1 rows);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q
index 3b585e7170c60f854be55e556454f96a4047d3b7..ac4b5242e1fc4a2c2622d7123c6a33c6439eeaee 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q
@@ -1,16 +1,18 @@
--- Conversion of main primitive types to String type: 
-SELECT CAST(NULL AS STRING) FROM src LIMIT 1;
+set hive.fetch.task.conversion=more;
 
-SELECT CAST(TRUE AS STRING) FROM src LIMIT 1;
+-- Conversion of main primitive types to String type:
+SELECT CAST(NULL AS STRING) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(1 AS TINYINT) AS STRING) FROM src LIMIT 1;
-SELECT CAST(CAST(-18 AS SMALLINT) AS STRING) FROM src LIMIT 1;
-SELECT CAST(-129 AS STRING) FROM src LIMIT 1;
-SELECT CAST(CAST(-1025 AS BIGINT) AS STRING) FROM src LIMIT 1;
+SELECT CAST(TRUE AS STRING) FROM src tablesample (1 rows);
 
-SELECT CAST(CAST(-3.14 AS DOUBLE) AS STRING) FROM src LIMIT 1;
-SELECT CAST(CAST(-3.14 AS FLOAT) AS STRING) FROM src LIMIT 1;
-SELECT CAST(CAST(-3.14 AS DECIMAL) AS STRING) FROM src LIMIT 1;
+SELECT CAST(CAST(1 AS TINYINT) AS STRING) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-18 AS SMALLINT) AS STRING) FROM src tablesample (1 rows);
+SELECT CAST(-129 AS STRING) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-1025 AS BIGINT) AS STRING) FROM src tablesample (1 rows);
 
-SELECT CAST('Foo' AS STRING) FROM src LIMIT 1;
+SELECT CAST(CAST(-3.14 AS DOUBLE) AS STRING) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-3.14 AS FLOAT) AS STRING) FROM src tablesample (1 rows);
+SELECT CAST(CAST(-3.14 AS DECIMAL(3,2)) AS STRING) FROM src tablesample (1 rows);
+
+SELECT CAST('Foo' AS STRING) FROM src tablesample (1 rows);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q
index 3024074bba4b3c1fdc58b660c63b7e29c286da73..0a2758edfcd4f7cddfef2359139043e18db39349 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q
@@ -1,8 +1,10 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION to_unix_timestamp;
 DESCRIBE FUNCTION EXTENDED to_unix_timestamp;
 
 create table oneline(key int, value string);
-load data local inpath '../data/files/things.txt' into table oneline;
+load data local inpath '../../data/files/things.txt' into table oneline;
 
 SELECT
   '2009-03-20 11:30:01',
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q
index cba6ff90f643fcb40dfb662cf3ffec1bcb9cea9a..21d799882f59bf8e98744c6d8c56e4912d49cbb9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION translate;
 DESCRIBE FUNCTION EXTENDED translate;
 
@@ -10,28 +12,28 @@ FROM src INSERT OVERWRITE TABLE table_translate SELECT 'abcd', 'ahd', '12' WHERE
 
 -- Run some queries on constant input parameters
 SELECT  translate('abcd', 'ab', '12'),
-        translate('abcd', 'abc', '12') FROM src LIMIT 1;
+        translate('abcd', 'abc', '12') FROM src tablesample (1 rows);
 
 -- Run some queries where first parameter being a table column while the other two being constants
 SELECT translate(table_input.input, 'ab', '12'),
-       translate(table_input.input, 'abc', '12') FROM table_input LIMIT 1;
+       translate(table_input.input, 'abc', '12') FROM table_input tablesample (1 rows);
 
 -- Run some queries where all parameters are coming from table columns
-SELECT translate(input_string, from_string, to_string) FROM table_translate LIMIT 1;
+SELECT translate(input_string, from_string, to_string) FROM table_translate tablesample (1 rows);
 
 -- Run some queries where some parameters are NULL
 SELECT translate(NULL, 'ab', '12'),
        translate('abcd', NULL, '12'),
        translate('abcd', 'ab', NULL),
-       translate(NULL, NULL, NULL) FROM src LIMIT 1;
+       translate(NULL, NULL, NULL) FROM src tablesample (1 rows);
 
 -- Run some queries where the same character appears several times in the from string (2nd argument) of the UDF
 SELECT translate('abcd', 'aba', '123'),
-       translate('abcd', 'aba', '12') FROM src LIMIT 1;
+       translate('abcd', 'aba', '12') FROM src tablesample (1 rows);
 
 -- Run some queries for the ignorant case when the 3rd parameter has more characters than the second one
-SELECT translate('abcd', 'abc', '1234') FROM src LIMIT 1;
+SELECT translate('abcd', 'abc', '1234') FROM src tablesample (1 rows);
 
 -- Test proper function over UTF-8 characters
-SELECT translate('Àbcd', 'À', 'Ã') FROM src LIMIT 1;
+SELECT translate('Àbcd', 'À', 'Ã') FROM src tablesample (1 rows);
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q
index e80021c8b2c51d9269ad9b1bd72daf9d9c913eb4..257e469ffb16cf5009b030e51b8a054aeaf23d25 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION unhex;
 DESCRIBE FUNCTION EXTENDED unhex;
 
@@ -9,11 +11,11 @@ SELECT
   unhex('61'),
   unhex('2D34'),
   unhex('')
-FROM src limit 1;
+FROM src tablesample (1 rows);
 
 -- Bad inputs
 SELECT
   unhex('MySQL'),
   unhex('G123'),
   unhex('\0')
-FROM src limit 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q
index 9140d22fb58c78a502d1e67f0f376220282f6acf..3876beb17d7ceeb1ee16084d58846639f3f9e5b3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q
@@ -1,11 +1,13 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION create_union;
 DESCRIBE FUNCTION EXTENDED create_union;
 
 EXPLAIN
 SELECT create_union(0, key), create_union(if(key<100, 0, 1), 2.0, value),
 create_union(1, "a", struct(2, "b"))
-FROM src LIMIT 2;
+FROM src tablesample (2 rows);
 
 SELECT create_union(0, key), create_union(if(key<100, 0, 1), 2.0, value),
 create_union(1, "a", struct(2, "b"))
-FROM src LIMIT 2;
+FROM src tablesample (2 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q
index 89288a1193c87a1ec745ee623affa1ae912c8b63..1664329c33d98fa37db0c4269e85a1bb4fdf53cc 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q
@@ -1,8 +1,10 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION unix_timestamp;
 DESCRIBE FUNCTION EXTENDED unix_timestamp;
 
 create table oneline(key int, value string);
-load data local inpath '../data/files/things.txt' into table oneline;
+load data local inpath '../../data/files/things.txt' into table oneline;
 
 SELECT
   '2009-03-20 11:30:01',
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_using.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_using.q
new file mode 100644
index 0000000000000000000000000000000000000000..093187ddc30cb981a90f7f450931d0f91cd7a56d
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_using.q
@@ -0,0 +1,15 @@
+dfs ${system:test.dfs.mkdir} hdfs:///tmp/udf_using;
+
+dfs -copyFromLocal ../../data/files/sales.txt hdfs:///tmp/udf_using/sales.txt;
+
+create function lookup as 'org.apache.hadoop.hive.ql.udf.UDFFileLookup' using file 'hdfs:///tmp/udf_using/sales.txt';
+
+create table udf_using (c1 string);
+insert overwrite table udf_using select 'Joe' from src limit 2;
+
+select c1, lookup(c1) from udf_using;
+
+drop table udf_using;
+drop function lookup;
+
+dfs -rmr hdfs:///tmp/udf_using;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q
index 4b7b4ea55aa8148ac4b53912df1909469a106dbb..abb0a2d7d2c086685e61b203d8a4e2513937384b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q
@@ -1,6 +1,8 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION weekofyear;
 DESCRIBE FUNCTION EXTENDED weekofyear;
 
 SELECT weekofyear('1980-01-01'), weekofyear('1980-01-06'), weekofyear('1980-01-07'), weekofyear('1980-12-31'),
 weekofyear('1984-1-1'), weekofyear('2008-02-20 00:00:00'), weekofyear('1980-12-28 23:59:59'), weekofyear('1980-12-29 23:59:59')
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q
index d94a594f026c57d59503209b514c32ae2ebad457..4eb7f6918a0dc0425f5bb1e85f60dbe118098a11 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION when;
 DESCRIBE FUNCTION EXTENDED when;
 
@@ -27,7 +29,7 @@ SELECT CASE
         WHEN 25=26 THEN 27
         WHEN 28=28 THEN NULL
        END
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
 
 SELECT CASE
         WHEN 1=1 THEN 2
@@ -54,4 +56,21 @@ SELECT CASE
         WHEN 25=26 THEN 27
         WHEN 28=28 THEN NULL
        END
-FROM src LIMIT 1;
+FROM src tablesample (1 rows);
+
+-- Allow compatible types to be used in return value
+SELECT CASE
+        WHEN 1=1 THEN 123.0BD
+        ELSE 0.0BD
+       END,
+       CASE
+        WHEN 1=1 THEN 123
+        WHEN 1=2 THEN 1.0
+        ELSE 222.02BD
+       END,
+       CASE
+        WHEN 1=1 THEN 'abcd'
+        WHEN 1=2 THEN cast('efgh' as varchar(10))
+        ELSE cast('ijkl' as char(4))
+       END
+FROM src tablesample (1 rows);
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q
index fca1ba11d85b2692efa0474dd3d107e4b09f011d..1ad38abcf6ec506c0736a697d691b96330d2a169 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q
@@ -1,8 +1,10 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION xpath ;
 DESCRIBE FUNCTION EXTENDED xpath ;
 
-SELECT xpath ('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>', 'a/text()') FROM src LIMIT 1 ;
-SELECT xpath ('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>', 'a/*/text()') FROM src LIMIT 1 ;
-SELECT xpath ('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>', 'a/b/text()') FROM src LIMIT 1 ;
-SELECT xpath ('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>', 'a/c/text()') FROM src LIMIT 1 ;
-SELECT xpath ('<a><b class="bb">b1</b><b>b2</b><b>b3</b><c class="bb">c1</c><c>c2</c></a>', 'a/*[@class="bb"]/text()') FROM src LIMIT 1 ;
\ No newline at end of file
+SELECT xpath ('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>', 'a/text()') FROM src tablesample (1 rows) ;
+SELECT xpath ('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>', 'a/*/text()') FROM src tablesample (1 rows) ;
+SELECT xpath ('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>', 'a/b/text()') FROM src tablesample (1 rows) ;
+SELECT xpath ('<a><b>b1</b><b>b2</b><b>b3</b><c>c1</c><c>c2</c></a>', 'a/c/text()') FROM src tablesample (1 rows) ;
+SELECT xpath ('<a><b class="bb">b1</b><b>b2</b><b>b3</b><c class="bb">c1</c><c>c2</c></a>', 'a/*[@class="bb"]/text()') FROM src tablesample (1 rows) ;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q
index 3a6e613eb6a0e9a1640c7a8576ab59f5ba9ae330..6e3ff244b04344e1e542cce62bdd64e5bd65bb06 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q
@@ -1,9 +1,11 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION xpath_boolean ;
 DESCRIBE FUNCTION EXTENDED xpath_boolean ;
 
-SELECT xpath_boolean ('<a><b>b</b></a>', 'a/b') FROM src LIMIT 1 ;
-SELECT xpath_boolean ('<a><b>b</b></a>', 'a/c') FROM src LIMIT 1 ;
-SELECT xpath_boolean ('<a><b>b</b></a>', 'a/b = "b"') FROM src LIMIT 1 ;
-SELECT xpath_boolean ('<a><b>b</b></a>', 'a/b = "c"') FROM src LIMIT 1 ;
-SELECT xpath_boolean ('<a><b>10</b></a>', 'a/b < 10') FROM src LIMIT 1 ;
-SELECT xpath_boolean ('<a><b>10</b></a>', 'a/b = 10') FROM src LIMIT 1 ;
+SELECT xpath_boolean ('<a><b>b</b></a>', 'a/b') FROM src tablesample (1 rows) ;
+SELECT xpath_boolean ('<a><b>b</b></a>', 'a/c') FROM src tablesample (1 rows) ;
+SELECT xpath_boolean ('<a><b>b</b></a>', 'a/b = "b"') FROM src tablesample (1 rows) ;
+SELECT xpath_boolean ('<a><b>b</b></a>', 'a/b = "c"') FROM src tablesample (1 rows) ;
+SELECT xpath_boolean ('<a><b>10</b></a>', 'a/b < 10') FROM src tablesample (1 rows) ;
+SELECT xpath_boolean ('<a><b>10</b></a>', 'a/b = 10') FROM src tablesample (1 rows) ;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q
index 4328747f444975ba112bcfdf0aa66c82b2cfec84..68441762e5a66ba57473766e5c017b030b67ad3c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q
@@ -1,14 +1,16 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION xpath_number ;
 DESCRIBE FUNCTION EXTENDED xpath_number ;
 
 DESCRIBE FUNCTION xpath_double ;
 DESCRIBE FUNCTION EXTENDED xpath_double ;
 
-SELECT xpath_double ('<a>this is not a number</a>', 'a') FROM src LIMIT 1 ;
-SELECT xpath_double ('<a>this 2 is not a number</a>', 'a') FROM src LIMIT 1 ;
-SELECT xpath_double ('<a><b>2000000000</b><c>40000000000</c></a>', 'a/b * a/c') FROM src LIMIT 1 ;
-SELECT xpath_double ('<a>try a boolean</a>', 'a = 10') FROM src LIMIT 1 ;
-SELECT xpath_double ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'a/b') FROM src LIMIT 1 ;
-SELECT xpath_double ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/*)') FROM src LIMIT 1 ;
-SELECT xpath_double ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b)') FROM src LIMIT 1 ;
-SELECT xpath_double ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ;
\ No newline at end of file
+SELECT xpath_double ('<a>this is not a number</a>', 'a') FROM src tablesample (1 rows) ;
+SELECT xpath_double ('<a>this 2 is not a number</a>', 'a') FROM src tablesample (1 rows) ;
+SELECT xpath_double ('<a><b>2000000000</b><c>40000000000</c></a>', 'a/b * a/c') FROM src tablesample (1 rows) ;
+SELECT xpath_double ('<a>try a boolean</a>', 'a = 10') FROM src tablesample (1 rows) ;
+SELECT xpath_double ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'a/b') FROM src tablesample (1 rows) ;
+SELECT xpath_double ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/*)') FROM src tablesample (1 rows) ;
+SELECT xpath_double ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b)') FROM src tablesample (1 rows) ;
+SELECT xpath_double ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q
index 1f1482833c858a45f9596152f1df7689bab39dd2..4596a322604a518eaa6153f20b9484ed6fda1473 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q
@@ -1,11 +1,13 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION xpath_float ;
 DESCRIBE FUNCTION EXTENDED xpath_float ;
 
-SELECT xpath_float ('<a>this is not a number</a>', 'a') FROM src LIMIT 1 ;
-SELECT xpath_float ('<a>this 2 is not a number</a>', 'a') FROM src LIMIT 1 ;
-SELECT xpath_float ('<a><b>2000000000</b><c>40000000000</c></a>', 'a/b * a/c') FROM src LIMIT 1 ;
-SELECT xpath_float ('<a>try a boolean</a>', 'a = 10') FROM src LIMIT 1 ;
-SELECT xpath_float ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'a/b') FROM src LIMIT 1 ;
-SELECT xpath_float ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/*)') FROM src LIMIT 1 ;
-SELECT xpath_float ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b)') FROM src LIMIT 1 ;
-SELECT xpath_float ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ;
\ No newline at end of file
+SELECT xpath_float ('<a>this is not a number</a>', 'a') FROM src tablesample (1 rows) ;
+SELECT xpath_float ('<a>this 2 is not a number</a>', 'a') FROM src tablesample (1 rows) ;
+SELECT xpath_float ('<a><b>2000000000</b><c>40000000000</c></a>', 'a/b * a/c') FROM src tablesample (1 rows) ;
+SELECT xpath_float ('<a>try a boolean</a>', 'a = 10') FROM src tablesample (1 rows) ;
+SELECT xpath_float ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'a/b') FROM src tablesample (1 rows) ;
+SELECT xpath_float ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/*)') FROM src tablesample (1 rows) ;
+SELECT xpath_float ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b)') FROM src tablesample (1 rows) ;
+SELECT xpath_float ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q
index 9b50bd9eeb3651e8c626f83b72f61d1d50eae018..9f3898f1147ac046f2a5703b3aaf3642c121f21b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q
@@ -1,11 +1,13 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION xpath_int ;
 DESCRIBE FUNCTION EXTENDED xpath_int ;
 
-SELECT xpath_int ('<a>this is not a number</a>', 'a') FROM src LIMIT 1 ;
-SELECT xpath_int ('<a>this 2 is not a number</a>', 'a') FROM src LIMIT 1 ;
-SELECT xpath_int ('<a><b>2000000000</b><c>40000000000</c></a>', 'a/b * a/c') FROM src LIMIT 1 ;
-SELECT xpath_int ('<a>try a boolean</a>', 'a = 10') FROM src LIMIT 1 ;
-SELECT xpath_int ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'a/b') FROM src LIMIT 1 ;
-SELECT xpath_int ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/*)') FROM src LIMIT 1 ;
-SELECT xpath_int ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b)') FROM src LIMIT 1 ;
-SELECT xpath_int ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ;
\ No newline at end of file
+SELECT xpath_int ('<a>this is not a number</a>', 'a') FROM src tablesample (1 rows) ;
+SELECT xpath_int ('<a>this 2 is not a number</a>', 'a') FROM src tablesample (1 rows) ;
+SELECT xpath_int ('<a><b>2000000000</b><c>40000000000</c></a>', 'a/b * a/c') FROM src tablesample (1 rows) ;
+SELECT xpath_int ('<a>try a boolean</a>', 'a = 10') FROM src tablesample (1 rows) ;
+SELECT xpath_int ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'a/b') FROM src tablesample (1 rows) ;
+SELECT xpath_int ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/*)') FROM src tablesample (1 rows) ;
+SELECT xpath_int ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b)') FROM src tablesample (1 rows) ;
+SELECT xpath_int ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q
index 04ee61b1f0e9d435b6c11bb57e525a9f439fc88a..3a335937c54fb4e593eff7c32e7a2438eec19d93 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q
@@ -1,11 +1,13 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION xpath_long ;
 DESCRIBE FUNCTION EXTENDED xpath_long ;
 
-SELECT xpath_long ('<a>this is not a number</a>', 'a') FROM src LIMIT 1 ;
-SELECT xpath_long ('<a>this 2 is not a number</a>', 'a') FROM src LIMIT 1 ;
-SELECT xpath_long ('<a><b>2000000000</b><c>40000000000</c></a>', 'a/b * a/c') FROM src LIMIT 1 ;
-SELECT xpath_long ('<a>try a boolean</a>', 'a = 10') FROM src LIMIT 1 ;
-SELECT xpath_long ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'a/b') FROM src LIMIT 1 ;
-SELECT xpath_long ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/*)') FROM src LIMIT 1 ;
-SELECT xpath_long ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b)') FROM src LIMIT 1 ;
-SELECT xpath_long ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ;
+SELECT xpath_long ('<a>this is not a number</a>', 'a') FROM src tablesample (1 rows) ;
+SELECT xpath_long ('<a>this 2 is not a number</a>', 'a') FROM src tablesample (1 rows) ;
+SELECT xpath_long ('<a><b>2000000000</b><c>40000000000</c></a>', 'a/b * a/c') FROM src tablesample (1 rows) ;
+SELECT xpath_long ('<a>try a boolean</a>', 'a = 10') FROM src tablesample (1 rows) ;
+SELECT xpath_long ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'a/b') FROM src tablesample (1 rows) ;
+SELECT xpath_long ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/*)') FROM src tablesample (1 rows) ;
+SELECT xpath_long ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b)') FROM src tablesample (1 rows) ;
+SELECT xpath_long ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q
index 6a1abdc0bdb215ef926a38222a9a936ac3cb3a9f..073056e72e7a1b6006b958c6f5772a61f89fc08b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q
@@ -1,11 +1,13 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION xpath_short ;
 DESCRIBE FUNCTION EXTENDED xpath_short ;
 
-SELECT xpath_short ('<a>this is not a number</a>', 'a') FROM src LIMIT 1 ;
-SELECT xpath_short ('<a>this 2 is not a number</a>', 'a') FROM src LIMIT 1 ;
-SELECT xpath_short ('<a><b>2000000000</b><c>40000000000</c></a>', 'a/b * a/c') FROM src LIMIT 1 ;
-SELECT xpath_short ('<a>try a boolean</a>', 'a = 10') FROM src LIMIT 1 ;
-SELECT xpath_short ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'a/b') FROM src LIMIT 1 ;
-SELECT xpath_short ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/*)') FROM src LIMIT 1 ;
-SELECT xpath_short ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b)') FROM src LIMIT 1 ;
-SELECT xpath_short ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ;
\ No newline at end of file
+SELECT xpath_short ('<a>this is not a number</a>', 'a') FROM src tablesample (1 rows) ;
+SELECT xpath_short ('<a>this 2 is not a number</a>', 'a') FROM src tablesample (1 rows) ;
+SELECT xpath_short ('<a><b>2000000000</b><c>40000000000</c></a>', 'a/b * a/c') FROM src tablesample (1 rows) ;
+SELECT xpath_short ('<a>try a boolean</a>', 'a = 10') FROM src tablesample (1 rows) ;
+SELECT xpath_short ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'a/b') FROM src tablesample (1 rows) ;
+SELECT xpath_short ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/*)') FROM src tablesample (1 rows) ;
+SELECT xpath_short ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b)') FROM src tablesample (1 rows) ;
+SELECT xpath_short ('<a><b class="odd">1</b><b class="even">2</b><b class="odd">4</b><c>8</c></a>', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q
index ebbc913b71bfbccd6ca60f9530d47e9b8a60fac9..1f1731c67d114d853cd8f31bbe2f232d3bd9766e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q
@@ -1,11 +1,13 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION xpath_string ;
 DESCRIBE FUNCTION EXTENDED xpath_string ;
 
-SELECT xpath_string ('<a><b>bb</b><c>cc</c></a>', 'a') FROM src LIMIT 1 ;
-SELECT xpath_string ('<a><b>bb</b><c>cc</c></a>', 'a/b') FROM src LIMIT 1 ;
-SELECT xpath_string ('<a><b>bb</b><c>cc</c></a>', 'a/c') FROM src LIMIT 1 ;
-SELECT xpath_string ('<a><b>bb</b><c>cc</c></a>', 'a/d') FROM src LIMIT 1 ;
-SELECT xpath_string ('<a><b>b1</b><b>b2</b></a>', '//b') FROM src LIMIT 1 ;
-SELECT xpath_string ('<a><b>b1</b><b>b2</b></a>', 'a/b[1]') FROM src LIMIT 1 ;
-SELECT xpath_string ('<a><b>b1</b><b>b2</b></a>', 'a/b[2]') FROM src LIMIT 1 ;
-SELECT xpath_string ('<a><b>b1</b><b id="b_2">b2</b></a>', 'a/b[@id="b_2"]') FROM src LIMIT 1 ;
+SELECT xpath_string ('<a><b>bb</b><c>cc</c></a>', 'a') FROM src tablesample (1 rows) ;
+SELECT xpath_string ('<a><b>bb</b><c>cc</c></a>', 'a/b') FROM src tablesample (1 rows) ;
+SELECT xpath_string ('<a><b>bb</b><c>cc</c></a>', 'a/c') FROM src tablesample (1 rows) ;
+SELECT xpath_string ('<a><b>bb</b><c>cc</c></a>', 'a/d') FROM src tablesample (1 rows) ;
+SELECT xpath_string ('<a><b>b1</b><b>b2</b></a>', '//b') FROM src tablesample (1 rows) ;
+SELECT xpath_string ('<a><b>b1</b><b>b2</b></a>', 'a/b[1]') FROM src tablesample (1 rows) ;
+SELECT xpath_string ('<a><b>b1</b><b>b2</b></a>', 'a/b[2]') FROM src tablesample (1 rows) ;
+SELECT xpath_string ('<a><b>b1</b><b id="b_2">b2</b></a>', 'a/b[@id="b_2"]') FROM src tablesample (1 rows) ;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q
index 638a4e9ca57e7984dfbd26ff81b32d68e8fb0255..1d405b3560c5cf822e0c6a3f2cb7cfdc2fece321 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q
@@ -1,3 +1,5 @@
+set hive.fetch.task.conversion=more;
+
 DESCRIBE FUNCTION explode;
 DESCRIBE FUNCTION EXTENDED explode;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q
index 712d9598c755dd62a454b696e3abad74bcb32217..93d829d4ed1752ac147c66aa703279da50da8b93 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q
@@ -2,17 +2,17 @@ create table json_t (key string, jstring string);
 
 insert overwrite table json_t
 select * from (
-  select '1', '{"f1": "value1", "f2": "value2", "f3": 3, "f5": 5.23}' from src limit 1
+  select '1', '{"f1": "value1", "f2": "value2", "f3": 3, "f5": 5.23}' from src tablesample (1 rows)
   union all
-  select '2', '{"f1": "value12", "f3": "value3", "f2": 2, "f4": 4.01}' from src limit 1
+  select '2', '{"f1": "value12", "f3": "value3", "f2": 2, "f4": 4.01}' from src tablesample (1 rows)
   union all
-  select '3', '{"f1": "value13", "f4": "value44", "f3": "value33", "f2": 2, "f5": 5.01}' from src limit 1
+  select '3', '{"f1": "value13", "f4": "value44", "f3": "value33", "f2": 2, "f5": 5.01}' from src tablesample (1 rows)
   union all
-  select '4', cast(null as string) from src limit 1
+  select '4', cast(null as string) from src tablesample (1 rows)
   union all
-  select '5', '{"f1": "", "f5": null}' from src limit 1
+  select '5', '{"f1": "", "f5": null}' from src tablesample (1 rows)
   union all
-  select '6', '[invalid JSON string]' from src limit 1
+  select '6', '[invalid JSON string]' from src tablesample (1 rows)
 ) s;
 
 explain 
@@ -40,7 +40,7 @@ select f2, count(*) from json_t a lateral view json_tuple(a.jstring, 'f1', 'f2',
 
 CREATE TABLE dest1(c1 STRING) STORED AS RCFILE;
 
-INSERT OVERWRITE TABLE dest1 SELECT '{"a":"b\nc"}' FROM src LIMIT 1;
+INSERT OVERWRITE TABLE dest1 SELECT '{"a":"b\nc"}' FROM src tablesample (1 rows);
 
 SELECT * FROM dest1;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q
index 055e39b2973a6a50c81e265de58a66c17f02f295..0870cbc4a9950d8b6499b481439bf75818dcd444 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q
@@ -2,17 +2,17 @@ create table url_t (key string, fullurl string);
 
 insert overwrite table url_t
 select * from (
-  select '1', 'http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1' from src limit 1
+  select '1', 'http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1' from src tablesample (1 rows)
   union all
-  select '2', 'https://www.socs.uts.edu.au:80/MosaicDocs-old/url-primer.html?k1=tps#chapter1' from src limit 1
+  select '2', 'https://www.socs.uts.edu.au:80/MosaicDocs-old/url-primer.html?k1=tps#chapter1' from src tablesample (1 rows)
   union all
-  select '3', 'ftp://sites.google.com/a/example.com/site/page' from src limit 1
+  select '3', 'ftp://sites.google.com/a/example.com/site/page' from src tablesample (1 rows)
   union all
-  select '4', cast(null as string) from src limit 1
+  select '4', cast(null as string) from src tablesample (1 rows)
   union all
-  select '5', 'htttp://' from src limit 1
+  select '5', 'htttp://' from src tablesample (1 rows)
   union all
-  select '6', '[invalid url string]' from src limit 1
+  select '6', '[invalid url string]' from src tablesample (1 rows)
 ) s;
 
 describe function parse_url_tuple;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_posexplode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_posexplode.q
new file mode 100644
index 0000000000000000000000000000000000000000..343f08ba6fe4baa640cc60c5cc3ead8cacc6a796
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_posexplode.q
@@ -0,0 +1,15 @@
+CREATE TABLE employees (
+name STRING,
+salary FLOAT,
+subordinates ARRAY<STRING>,
+deductions MAP<STRING, FLOAT>,
+address STRUCT<street:STRING, city:STRING, state:STRING, zip:INT>);
+
+LOAD DATA LOCAL INPATH '../../data/files/posexplode_data.txt' INTO TABLE employees;
+
+SELECT
+  name, pos, sub
+FROM
+  employees
+LATERAL VIEW
+  posexplode(subordinates) subView AS pos, sub;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q
index 91bbd1b07f08b0c737e731d5e49df42e252d3dea..525eccbbfe22753956fe031efb031db08a9d4139 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q
@@ -6,13 +6,13 @@ FROM (
   UNION ALL
   FROM src SELECT src.* WHERE src.key > 100
 ) unioninput
-INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.*;
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*;
 
 FROM (
   FROM src select src.key, src.value WHERE src.key < 100
   UNION ALL
   FROM src SELECT src.* WHERE src.key > 100
 ) unioninput
-INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.*;
+INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*;
 
-dfs -cat ../build/ql/test/data/warehouse/union.out/*;
+dfs -cat ${system:test.warehouse.dir}/union.out/*;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q
index 36bc865be81febf3a84cf18cc4109e108149ffae..238b583fd86a83297917a6169524374ff6f7963f 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q
@@ -3,7 +3,7 @@ create table src10_2 (key string, value string);
 create table src10_3 (key string, value string);
 create table src10_4 (key string, value string);
 
-from (select * from src limit 10) a
+from (select * from src tablesample (10 rows)) a
 insert overwrite table src10_1 select *
 insert overwrite table src10_2 select *
 insert overwrite table src10_3 select *
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q
index e332a8af7bb1b1923af77d58f61416e4270d2903..dd6f08e5b4e21a4264f1f226dd053198d7ab004b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q
@@ -17,8 +17,8 @@ create table union_date_2 (
   FL_NUM int
 );
 
-LOAD DATA LOCAL INPATH '../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_1;
-LOAD DATA LOCAL INPATH '../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_2;
+LOAD DATA LOCAL INPATH '../../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_1;
+LOAD DATA LOCAL INPATH '../../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_2;
 
 select * from (
   select fl_num, fl_date from union_date_1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q
index 70147070feacaacec570ac2c13129c7608dc47a1..4368b8a5b6c363c090a2ea61feb67793eec26e8d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q
@@ -1,2 +1,5 @@
 -- HIVE-2901
 select x from (select value as x from src union all select NULL as x from src)a limit 10;
+
+-- HIVE-4837
+select * from (select null as N from src1 group by key UNION ALL select null as N from src1 group by key ) a;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q
index c9f920c648a9249ae703209eb1b38cd01b8a49c4..c87b3fef1e4629065099f5e312010facb4118a91 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q
@@ -19,7 +19,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as textfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q
index a072fe3b56e435a5485948788b29c92b3f2c94d2..6701952717eb59ae5dea302dce1f9e6afb1d64eb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q
@@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as rcfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q
index 6250a20210f9f357aff39c84d8b09bc1c3b9fa58..4b2fa42f1b54d2140eda3bf4110cd781bbf35f31 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q
@@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as rcfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q
index 168eac34a03013bb2dd06d304dbbafe571a73097..69d0d0af9ff68ed01866f5d9d43d00f3d9d15dca 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q
@@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as rcfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q
index a88a83e152a5f772bb8691da84cb0af0d808e1f8..7605f0ec26a089c3b9014690a87220f6f9035363 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q
@@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as rcfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q
index e588e8fd62ab9d91fa785cc274a267a3962a71b9..a4fdfc8ee44ef5ed2f9f01109f0b55c97c0442c7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q
@@ -25,7 +25,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as rcfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q
index 237f2e7629057c800960be5621c0d7353f9dd560..e3c937b7b1e067e2d987c6eb670660e73204d622 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q
@@ -25,7 +25,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1 partition (ds)
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q
index 06d5043f9cf9dbfd0da679fbf35a6a27530f9d8f..537078b080433c6947b7b6b028d8c5c5891765ec 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q
@@ -25,7 +25,7 @@ set hive.exec.dynamic.partition=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile ;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1 partition (ds)
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q
index 65b8255c3e954c112995df9f53991f04cef35f35..d70f3d3da31a233ebe9e4cc9055afa45e66be233 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q
@@ -22,7 +22,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1 partition (ds)
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q
index 265acfd35758f6dc0d2ba4f18c7147b6bb656f4c..478650038cc9c4dfed0221b2b88270887cc9d983 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q
@@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, ds string) stored as textfile;
 create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1 partition (ds)
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q
index 1450caa2d07a7e82ac4d4a9c5db078b4172e2f30..8c45953ed877c4b0a431c35bf6306065c6c829d0 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q
@@ -19,7 +19,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as textfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q
index 015c146ecadf04f5e515942b3126b3ae926e368d..83cd2887ab23f33d86ebcdbea5efa39a43c2d3f4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q
@@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as textfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q
index ac727889273d7efb0dcd571455608d629014bd20..f80f7c1dfdaddfb95e21a70ea64bc340d40998a3 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q
@@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(values bigint, key string) stored as textfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q
index f89744624116fdcc0b3d560bd2bf66e1f437a593..8963c2576ca760b86abf762634713278ac5c5377 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q
@@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string) stored as textfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q
index f01053d27d0c0559cd2fbca8ed90bbe44b2b6318..b0c1ccd73db687891a87f45232e6004b5141b946 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q
@@ -19,7 +19,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint, values2 bigint) stored as textfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q
index 805dd76274c4211c18a5ff8804babdf8fc252a18..a1b989a0eb63ca048d603e8a52c566eacaf1e506 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q
@@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as textfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q
index 36fd947d6035d98b3803dced1db69b4f8a594b2b..ec561e0979e0c4f9fd3d37e1853d3a9643d046f1 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q
@@ -18,7 +18,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key double, values bigint) stored as textfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 EXPLAIN
 INSERT OVERWRITE TABLE outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q
index da0f1c0473ffe947c09ec3006ddc768a1d40682c..9617f737bc55222f69c8fc40932aed2da370a0fb 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q
@@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as textfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q
index 18d4730a120240ce34f1f60b257b2baa3cb93781..cae323b1ef89d94ad74b053749bfb60bb2761ac7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q
@@ -20,7 +20,7 @@ set hive.merge.smallfiles.avgsize=1;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as textfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q
index a6fbeb03ddcc3d1ee9309de9aad908cc3b000ea0..5df84e145c709ed84cb4c6c551d5ae2888ca9f48 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q
@@ -22,7 +22,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as textfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q
index 7ae5af30c1baea60519d6a9fab9fbc14ac037161..bfce26d0fbe1fced4ca65bcba90e6f71f33ba2f7 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q
@@ -17,7 +17,7 @@ create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as textfile;
 create table outputTbl2(key string, values bigint) stored as textfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 FROM (
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q
index 5a639ca11700dfc1bfe380bb02d56b2de74294d2..3a956747a473d4243041513039634b3d164799bd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q
@@ -21,7 +21,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as rcfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q
index 79b84e81181d3439bd3386dc61217cc2bacf950b..a83a43e466f550ed4072ce496c26f5b6783e3bb6 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q
@@ -22,7 +22,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as rcfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q
index f6038384f3fa8d179fa8936fd02cc924085fd60a..e71f6dd001dd63262181e208cd7e758ab5bd34e5 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q
@@ -22,7 +22,7 @@ set mapred.input.dir.recursive=true;
 create table inputTbl1(key string, val string) stored as textfile;
 create table outputTbl1(key string, values bigint) stored as rcfile;
 
-load data local inpath '../data/files/T1.txt' into table inputTbl1;
+load data local inpath '../../data/files/T1.txt' into table inputTbl1;
 
 explain
 insert overwrite table outputTbl1
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_top_level.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_top_level.q
new file mode 100644
index 0000000000000000000000000000000000000000..6a4b45fbc45998d63bec318c7166fd2db1549aa0
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_top_level.q
@@ -0,0 +1,106 @@
+-- top level
+explain
+select * from (
+select key, 0 as value from src where key % 3 == 0 limit 3
+union all
+select key, 1 as value from src where key % 3 == 1 limit 3
+union all
+select key, 2 as value from src where key % 3 == 2 limit 3
+) q1 order by key;
+
+select * from (
+select key, 0 as value from src where key % 3 == 0 limit 3
+union all
+select key, 1 as value from src where key % 3 == 1 limit 3
+union all
+select key, 2 as value from src where key % 3 == 2 limit 3
+) q1 order by key;
+
+explain
+select * from (
+select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10
+union all
+select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10
+) q1 order by k;
+
+select * from (
+select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10
+union all
+select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10
+) q1 order by k;
+
+-- ctas
+explain
+create table union_top as
+select key, 0 as value from src where key % 3 == 0 limit 3
+union all
+select key, 1 as value from src where key % 3 == 1 limit 3
+union all
+select key, 2 as value from src where key % 3 == 2 limit 3;
+
+create table union_top as
+select key, 0 as value from src where key % 3 == 0 limit 3
+union all
+select key, 1 as value from src where key % 3 == 1 limit 3
+union all
+select key, 2 as value from src where key % 3 == 2 limit 3;
+
+select * from union_top order by key;
+
+truncate table union_top;
+
+-- insert into
+explain
+insert into table union_top
+select key, 0 as value from src where key % 3 == 0 limit 3
+union all
+select key, 1 as value from src where key % 3 == 1 limit 3
+union all
+select key, 2 as value from src where key % 3 == 2 limit 3;
+
+insert into table union_top
+select key, 0 as value from src where key % 3 == 0 limit 3
+union all
+select key, 1 as value from src where key % 3 == 1 limit 3
+union all
+select key, 2 as value from src where key % 3 == 2 limit 3;
+
+select * from union_top order by key;
+
+explain
+insert overwrite table union_top
+select key, 0 as value from src where key % 3 == 0 limit 3
+union all
+select key, 1 as value from src where key % 3 == 1 limit 3
+union all
+select key, 2 as value from src where key % 3 == 2 limit 3;
+
+insert overwrite table union_top
+select key, 0 as value from src where key % 3 == 0 limit 3
+union all
+select key, 1 as value from src where key % 3 == 1 limit 3
+union all
+select key, 2 as value from src where key % 3 == 2 limit 3;
+
+select * from union_top order by key;
+
+-- create view
+explain
+create view union_top_view as
+select key, 0 as value from src where key % 3 == 0 limit 3
+union all
+select key, 1 as value from src where key % 3 == 1 limit 3
+union all
+select key, 2 as value from src where key % 3 == 2 limit 3;
+
+create view union_top_view as
+select key, 0 as value from src where key % 3 == 0 limit 3
+union all
+select key, 1 as value from src where key % 3 == 1 limit 3
+union all
+select key, 2 as value from src where key % 3 == 2 limit 3;
+
+select * from union_top_view order by key;
+
+drop table union_top;
+drop view union_top_view;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q
index 4f8bafe37a809632db4fd2706064908eb90a8a83..b727199cf4f250e022a2d3edb28abbbfb3ce0243 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q
@@ -1,3 +1,4 @@
+set hive.stats.dbclass=fs;
 CREATE TABLE src_union_1 (key int, value string) PARTITIONED BY (ds string);
 CREATE INDEX src_union_1_key_idx ON TABLE src_union_1(key) AS 'COMPACT' WITH DEFERRED REBUILD;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q
index 51bcf22bfa0be01e8e44e3c98f7702b7eb8a2563..3bc8ef931bcd39b9fbdb499d72ae605a46336bcd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q
@@ -2,9 +2,9 @@ CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE;
 CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE;
 CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1;
-LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2;
-LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3;
+LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1;
+LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2;
+LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3;
 
 FROM UNIQUEJOIN PRESERVE T1 a (a.key), PRESERVE T2 b (b.key), PRESERVE T3 c (c.key)
 SELECT a.key, b.key, c.key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q
index 53273b3187a78df0b78ab4c97a9d7df3068a482b..bed61f381ce96a9f5f6898d9e865dbf02607417e 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q
@@ -5,7 +5,7 @@ create table varchar1 (key varchar(10), value varchar(20));
 create table varchar1_1 (key string, value string);
 
 -- load from file
-load data local inpath '../data/files/srcbucket0.txt' overwrite into table varchar1;
+load data local inpath '../../data/files/srcbucket0.txt' overwrite into table varchar1;
 select * from varchar1 order by key, value limit 2;
 
 -- insert overwrite, from same/different length varchar
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q
index 550f3dc728df25036c5b41cacef1346b6434b563..c356b1dbca6ab72f32730104b8b45205484a50a9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q
@@ -1,3 +1,4 @@
+set hive.fetch.task.conversion=more;
 
 -- Cast from varchar to other data types
 select
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q
index b6c6f406412fea8037b2ef2187c6d9a2812135fe..05cad852a268067a3f7e9f56d0a9638437ad4821 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q
@@ -1,3 +1,4 @@
+set hive.fetch.task.conversion=more;
 
 -- Should all be true
 select
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q
index 6a19efaa3c5a13b348b09fe0ac6e3ad95b538c56..94226879c53ab138109a9166e8d317676e29864b 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q
@@ -17,9 +17,9 @@ create table  varchar_join1_str (
   c2 string
 );
 
-load data local inpath '../data/files/vc1.txt' into table varchar_join1_vc1;
-load data local inpath '../data/files/vc1.txt' into table varchar_join1_vc2;
-load data local inpath '../data/files/vc1.txt' into table varchar_join1_str;
+load data local inpath '../../data/files/vc1.txt' into table varchar_join1_vc1;
+load data local inpath '../../data/files/vc1.txt' into table varchar_join1_vc2;
+load data local inpath '../../data/files/vc1.txt' into table varchar_join1_str;
 
 -- Join varchar with same length varchar
 select * from varchar_join1_vc1 a join varchar_join1_vc1 b on (a.c2 = b.c2) order by a.c1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q
index 7351b688cf504e91fdafe4233253eabbada54a5b..ea2a022b940377ba5ba1a7634991b912380eec2d 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q
@@ -18,7 +18,7 @@ with serdeproperties (
 )
 stored as textfile;
 
-load data local inpath '../data/files/srcbucket0.txt' overwrite into table varchar_serde_regex;
+load data local inpath '../../data/files/srcbucket0.txt' overwrite into table varchar_serde_regex;
 
 select * from varchar_serde_regex limit 5;
 select value, count(*) from varchar_serde_regex group by value limit 5;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q
index cf90eab33e2d0052c646a0722a0ba20881358ff4..dd3cffe0db4fecdb83dae53c22d12907c81f2e84 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q
@@ -17,9 +17,9 @@ create table  varchar_union1_str (
   c2 string
 );
 
-load data local inpath '../data/files/vc1.txt' into table varchar_union1_vc1;
-load data local inpath '../data/files/vc1.txt' into table varchar_union1_vc2;
-load data local inpath '../data/files/vc1.txt' into table varchar_union1_str;
+load data local inpath '../../data/files/vc1.txt' into table varchar_union1_vc1;
+load data local inpath '../../data/files/vc1.txt' into table varchar_union1_vc2;
+load data local inpath '../../data/files/vc1.txt' into table varchar_union1_str;
 
 -- union varchar with same length varchar
 select * from (
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_between_in.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_between_in.q
new file mode 100644
index 0000000000000000000000000000000000000000..1bc66118f8ff3adb22578c450423c4fb95e8518e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_between_in.q
@@ -0,0 +1,35 @@
+SET hive.vectorized.execution.enabled=true;
+
+CREATE TABLE decimal_date_test STORED AS ORC AS SELECT cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2, CAST(CAST((CAST(cint AS BIGINT) *ctinyint) AS TIMESTAMP) AS DATE) AS cdate FROM alltypesorc ORDER BY cdate;
+
+EXPLAIN SELECT cdate FROM decimal_date_test WHERE cdate IN (CAST("1969-10-26" AS DATE), CAST("1969-07-14" AS DATE)) ORDER BY cdate;
+
+EXPLAIN SELECT COUNT(*) FROM decimal_date_test WHERE cdate NOT IN (CAST("1969-10-26" AS DATE), CAST("1969-07-14" AS DATE), CAST("1970-01-21" AS DATE));
+
+EXPLAIN SELECT cdecimal1 FROM decimal_date_test WHERE cdecimal1 IN (2365.8945945946, 881.0135135135, -3367.6517567568) ORDER BY cdecimal1;
+
+EXPLAIN SELECT COUNT(*) FROM decimal_date_test WHERE cdecimal1 NOT IN (2365.8945945946, 881.0135135135, -3367.6517567568);
+
+EXPLAIN SELECT cdate FROM decimal_date_test WHERE cdate BETWEEN CAST("1969-12-30" AS DATE) AND CAST("1970-01-02" AS DATE) ORDER BY cdate;
+
+EXPLAIN SELECT cdate FROM decimal_date_test WHERE cdate NOT BETWEEN CAST("1968-05-01" AS DATE) AND CAST("1971-09-01" AS DATE) ORDER BY cdate;
+
+EXPLAIN SELECT cdecimal1 FROM decimal_date_test WHERE cdecimal1 BETWEEN -20 AND 45.9918918919 ORDER BY cdecimal1;
+
+EXPLAIN SELECT COUNT(*) FROM decimal_date_test WHERE cdecimal1 NOT BETWEEN -2000 AND 4390.1351351351;
+
+SELECT cdate FROM decimal_date_test WHERE cdate IN (CAST("1969-10-26" AS DATE), CAST("1969-07-14" AS DATE)) ORDER BY cdate;
+
+SELECT COUNT(*) FROM decimal_date_test WHERE cdate NOT IN (CAST("1969-10-26" AS DATE), CAST("1969-07-14" AS DATE), CAST("1970-01-21" AS DATE));
+
+SELECT cdecimal1 FROM decimal_date_test WHERE cdecimal1 IN (2365.8945945946, 881.0135135135, -3367.6517567568) ORDER BY cdecimal1;
+
+SELECT COUNT(*) FROM decimal_date_test WHERE cdecimal1 NOT IN (2365.8945945946, 881.0135135135, -3367.6517567568);
+
+SELECT cdate FROM decimal_date_test WHERE cdate BETWEEN CAST("1969-12-30" AS DATE) AND CAST("1970-01-02" AS DATE) ORDER BY cdate;
+
+SELECT cdate FROM decimal_date_test WHERE cdate NOT BETWEEN CAST("1968-05-01" AS DATE) AND CAST("1971-09-01" AS DATE) ORDER BY cdate;
+
+SELECT cdecimal1 FROM decimal_date_test WHERE cdecimal1 BETWEEN -20 AND 45.9918918919 ORDER BY cdecimal1;
+
+SELECT COUNT(*) FROM decimal_date_test WHERE cdecimal1 NOT BETWEEN -2000 AND 4390.1351351351;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_coalesce.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_coalesce.q
new file mode 100644
index 0000000000000000000000000000000000000000..052ab716bee1a4c21fb9dd8b46e5740f51b5282c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_coalesce.q
@@ -0,0 +1,32 @@
+SET hive.vectorized.execution.enabled=true;
+EXPLAIN SELECT cdouble, cstring1, cint, cfloat, csmallint, coalesce(cdouble, cstring1, cint, cfloat, csmallint) 
+FROM alltypesorc
+WHERE (cdouble IS NULL) LIMIT 10;
+
+SELECT cdouble, cstring1, cint, cfloat, csmallint, coalesce(cdouble, cstring1, cint, cfloat, csmallint) 
+FROM alltypesorc
+WHERE (cdouble IS NULL) LIMIT 10;
+
+EXPLAIN SELECT ctinyint, cdouble, cint, coalesce(ctinyint+10, (cdouble+log2(cint)), 0) 
+FROM alltypesorc
+WHERE (ctinyint IS NULL) LIMIT 10;
+
+SELECT ctinyint, cdouble, cint, coalesce(ctinyint+10, (cdouble+log2(cint)), 0) 
+FROM alltypesorc
+WHERE (ctinyint IS NULL) LIMIT 10;
+
+EXPLAIN SELECT cfloat, cbigint, coalesce(cfloat, cbigint, 0) 
+FROM alltypesorc
+WHERE (cfloat IS NULL AND cbigint IS NULL) LIMIT 10;
+
+SELECT cfloat, cbigint, coalesce(cfloat, cbigint, 0) 
+FROM alltypesorc
+WHERE (cfloat IS NULL AND cbigint IS NULL) LIMIT 10;
+
+EXPLAIN SELECT ctimestamp1, ctimestamp2, coalesce(ctimestamp1, ctimestamp2) 
+FROM alltypesorc 
+WHERE ctimestamp1 IS NOT NULL OR ctimestamp2 IS NOT NULL LIMIT 10;
+
+SELECT ctimestamp1, ctimestamp2, coalesce(ctimestamp1, ctimestamp2) 
+FROM alltypesorc 
+WHERE ctimestamp1 IS NOT NULL OR ctimestamp2 IS NOT NULL LIMIT 10;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_aggregate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_aggregate.q
new file mode 100644
index 0000000000000000000000000000000000000000..eb9146e95d437f8c0cb1b54de775fdb73a8618e9
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_aggregate.q
@@ -0,0 +1,20 @@
+CREATE TABLE decimal_vgby STORED AS ORC AS 
+	SELECT cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, 
+	CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2,
+	cint
+	FROM alltypesorc;
+
+SET hive.vectorized.execution.enabled=true;
+
+EXPLAIN SELECT cint,
+	COUNT(cdecimal1), MAX(cdecimal1), MIN(cdecimal1), SUM(cdecimal1), AVG(cdecimal1), STDDEV_POP(cdecimal1), STDDEV_SAMP(cdecimal1),
+	COUNT(cdecimal2), MAX(cdecimal2), MIN(cdecimal2), SUM(cdecimal2), AVG(cdecimal2), STDDEV_POP(cdecimal2), STDDEV_SAMP(cdecimal2)
+	FROM decimal_vgby
+	GROUP BY cint
+	HAVING COUNT(*) > 1;
+SELECT cint,
+	COUNT(cdecimal1), MAX(cdecimal1), MIN(cdecimal1), SUM(cdecimal1), AVG(cdecimal1), STDDEV_POP(cdecimal1), STDDEV_SAMP(cdecimal1),
+	COUNT(cdecimal2), MAX(cdecimal2), MIN(cdecimal2), SUM(cdecimal2), AVG(cdecimal2), STDDEV_POP(cdecimal2), STDDEV_SAMP(cdecimal2)
+	FROM decimal_vgby
+	GROUP BY cint
+	HAVING COUNT(*) > 1;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_cast.q
new file mode 100644
index 0000000000000000000000000000000000000000..ea7a5b817d7c0b8043d8dd9e25368f4721468d7e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_cast.q
@@ -0,0 +1,5 @@
+SET hive.vectorized.execution.enabled=true;
+
+EXPLAIN SELECT cdouble, cint, cboolean1, ctimestamp1, CAST(cdouble AS DECIMAL(20,10)), CAST(cint AS DECIMAL(23,14)), CAST(cboolean1 AS DECIMAL(5,2)), CAST(ctimestamp1 AS DECIMAL(15,0)) FROM alltypesorc WHERE cdouble IS NOT NULL AND cint IS NOT NULL AND cboolean1 IS NOT NULL AND ctimestamp1 IS NOT NULL LIMIT 10;
+
+SELECT cdouble, cint, cboolean1, ctimestamp1, CAST(cdouble AS DECIMAL(20,10)), CAST(cint AS DECIMAL(23,14)), CAST(cboolean1 AS DECIMAL(5,2)), CAST(ctimestamp1 AS DECIMAL(15,0)) FROM alltypesorc WHERE cdouble IS NOT NULL AND cint IS NOT NULL AND cboolean1 IS NOT NULL AND ctimestamp1 IS NOT NULL LIMIT 10;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_expressions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_expressions.q
new file mode 100644
index 0000000000000000000000000000000000000000..a74b17bd127d6395c6aac1822690ca4ad4fa1881
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_expressions.q
@@ -0,0 +1,5 @@
+CREATE TABLE decimal_test STORED AS ORC AS SELECT cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2 FROM alltypesorc;
+SET hive.vectorized.execution.enabled=true;
+EXPLAIN SELECT cdecimal1 + cdecimal2, cdecimal1 - (2*cdecimal2), ((cdecimal1+2.34)/cdecimal2), (cdecimal1 * (cdecimal2/3.4)), cdecimal1 % 10, CAST(cdecimal1 AS INT), CAST(cdecimal2 AS SMALLINT), CAST(cdecimal2 AS TINYINT), CAST(cdecimal1 AS BIGINT), CAST (cdecimal1 AS BOOLEAN), CAST(cdecimal2 AS DOUBLE), CAST(cdecimal1 AS FLOAT), CAST(cdecimal2 AS STRING), CAST(cdecimal1 AS TIMESTAMP) FROM decimal_test WHERE cdecimal1 > 0 AND cdecimal1 < 12345.5678 AND cdecimal2 != 0 AND cdecimal2 > 1000 AND cdouble IS NOT NULL LIMIT 10;
+
+SELECT cdecimal1 + cdecimal2, cdecimal1 - (2*cdecimal2), ((cdecimal1+2.34)/cdecimal2), (cdecimal1 * (cdecimal2/3.4)), cdecimal1 % 10, CAST(cdecimal1 AS INT), CAST(cdecimal2 AS SMALLINT), CAST(cdecimal2 AS TINYINT), CAST(cdecimal1 AS BIGINT), CAST (cdecimal1 AS BOOLEAN), CAST(cdecimal2 AS DOUBLE), CAST(cdecimal1 AS FLOAT), CAST(cdecimal2 AS STRING), CAST(cdecimal1 AS TIMESTAMP) FROM decimal_test WHERE cdecimal1 > 0 AND cdecimal1 < 12345.5678 AND cdecimal2 != 0 AND cdecimal2 > 1000 AND cdouble IS NOT NULL LIMIT 10;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_mapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_mapjoin.q
new file mode 100644
index 0000000000000000000000000000000000000000..d8b3d1a9ac13fed07c0bb4815962d144f68fe9a0
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_mapjoin.q
@@ -0,0 +1,19 @@
+CREATE TABLE decimal_mapjoin STORED AS ORC AS 
+  SELECT cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, 
+  CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2,
+  cint
+  FROM alltypesorc;
+ 
+SET hive.auto.convert.join=true;
+SET hive.auto.convert.join.nonconditionaltask=true;
+SET hive.auto.convert.join.nonconditionaltask.size=1000000000;
+SET hive.vectorized.execution.enabled=true;
+
+EXPLAIN SELECT l.cint, r.cint, l.cdecimal1, r.cdecimal2
+  FROM decimal_mapjoin l
+  JOIN decimal_mapjoin r ON l.cint = r.cint
+  WHERE l.cint = 6981;
+SELECT l.cint, r.cint, l.cdecimal1, r.cdecimal2
+  FROM decimal_mapjoin l
+  JOIN decimal_mapjoin r ON l.cint = r.cint
+  WHERE l.cint = 6981;
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_math_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_math_funcs.q
new file mode 100644
index 0000000000000000000000000000000000000000..6e2c0b1fd403f4e57bec5b07522d3a3ad5374031
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_math_funcs.q
@@ -0,0 +1,77 @@
+CREATE TABLE decimal_test STORED AS ORC AS SELECT cbigint, cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2 FROM alltypesorc;
+SET hive.vectorized.execution.enabled=true;
+
+-- Test math functions in vectorized mode to verify they run correctly end-to-end.
+
+explain 
+select
+   cdecimal1
+  ,Round(cdecimal1, 2)
+  ,Round(cdecimal1)
+  ,Floor(cdecimal1)
+  ,Ceil(cdecimal1)
+  ,Exp(cdecimal1)
+  ,Ln(cdecimal1)  
+  ,Log10(cdecimal1)
+  -- Use log2 as a representative function to test all input types.
+  ,Log2(cdecimal1)
+  -- Use 15601.0 to test zero handling, as there are no zeroes in the table
+  ,Log2(cdecimal1 - 15601.0)
+  ,Log(2.0, cdecimal1)
+  ,Pow(log2(cdecimal1), 2.0)  
+  ,Power(log2(cdecimal1), 2.0)
+  ,Sqrt(cdecimal1)
+  ,Abs(cdecimal1)
+  ,Sin(cdecimal1)
+  ,Asin(cdecimal1)
+  ,Cos(cdecimal1)
+  ,ACos(cdecimal1)
+  ,Atan(cdecimal1)
+  ,Degrees(cdecimal1)
+  ,Radians(cdecimal1)
+  ,Positive(cdecimal1)
+  ,Negative(cdecimal1)
+  ,Sign(cdecimal1)
+  -- Test nesting
+  ,cos(-sin(log(cdecimal1)) + 3.14159)
+from decimal_test
+-- limit output to a reasonably small number of rows
+where cbigint % 500 = 0
+-- test use of a math function in the WHERE clause
+and sin(cdecimal1) >= -1.0;
+
+select
+   cdecimal1
+  ,Round(cdecimal1, 2)
+  ,Round(cdecimal1)
+  ,Floor(cdecimal1)
+  ,Ceil(cdecimal1)
+  ,Exp(cdecimal1)
+  ,Ln(cdecimal1)  
+  ,Log10(cdecimal1)
+  -- Use log2 as a representative function to test all input types.
+  ,Log2(cdecimal1)
+  -- Use 15601.0 to test zero handling, as there are no zeroes in the table
+  ,Log2(cdecimal1 - 15601.0)
+  ,Log(2.0, cdecimal1)
+  ,Pow(log2(cdecimal1), 2.0)  
+  ,Power(log2(cdecimal1), 2.0)
+  ,Sqrt(cdecimal1)
+  ,Abs(cdecimal1)
+  ,Sin(cdecimal1)
+  ,Asin(cdecimal1)
+  ,Cos(cdecimal1)
+  ,ACos(cdecimal1)
+  ,Atan(cdecimal1)
+  ,Degrees(cdecimal1)
+  ,Radians(cdecimal1)
+  ,Positive(cdecimal1)
+  ,Negative(cdecimal1)
+  ,Sign(cdecimal1)
+  -- Test nesting
+  ,cos(-sin(log(cdecimal1)) + 3.14159)
+from decimal_test
+-- limit output to a reasonably small number of rows
+where cbigint % 500 = 0
+-- test use of a math function in the WHERE clause
+and sin(cdecimal1) >= -1.0;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_left_outer_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_left_outer_join.q
new file mode 100644
index 0000000000000000000000000000000000000000..6e966904973903fe1f03a5b7d6d84440a8763c7f
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_left_outer_join.q
@@ -0,0 +1,21 @@
+set hive.vectorized.execution.enabled=true;
+set hive.auto.convert.join=true;
+explain 
+select count(*) from (select c.ctinyint 
+from alltypesorc c
+left outer join alltypesorc cd
+  on cd.cint = c.cint 
+left outer join alltypesorc hd
+  on hd.ctinyint = c.ctinyint
+) t1
+;
+select count(*) from (select c.ctinyint
+from alltypesorc c
+left outer join alltypesorc cd
+  on cd.cint = c.cint 
+left outer join alltypesorc hd
+  on hd.ctinyint = c.ctinyint
+) t1;
+
+set hive.auto.convert.join=false;
+set hive.vectorized.execution.enabled=false;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_non_string_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_non_string_partition.q
new file mode 100644
index 0000000000000000000000000000000000000000..fc1dc6d3b89f31b48131c25df4a04eccd8829ffa
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_non_string_partition.q
@@ -0,0 +1,17 @@
+SET hive.vectorized.execution.enabled=true;
+CREATE TABLE non_string_part(cint INT, cstring1 STRING, cdouble DOUBLE, ctimestamp1 TIMESTAMP) PARTITIONED BY (ctinyint tinyint) STORED AS ORC;
+SET hive.exec.dynamic.partition.mode=nonstrict;
+SET hive.exec.dynamic.partition=true;
+
+INSERT OVERWRITE TABLE non_string_part PARTITION(ctinyint) SELECT cint, cstring1, cdouble, ctimestamp1, ctinyint fROM alltypesorc 
+WHERE ctinyint IS NULL AND cdouble IS NOT NULL ORDER BY cdouble;
+
+SHOW PARTITIONS non_string_part;
+
+EXPLAIN SELECT cint, ctinyint FROM non_string_part WHERE cint > 0 ORDER BY cint LIMIT 10;
+
+SELECT cint, ctinyint FROM non_string_part WHERE cint > 0 ORDER BY cint LIMIT 10;
+
+EXPLAIN SELECT cint, cstring1 FROM non_string_part WHERE cint > 0 ORDER BY cint, cstring1 LIMIT 10;
+
+SELECT cint, cstring1 FROM non_string_part WHERE cint > 0 ORDER BY cint, cstring1 LIMIT 10;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_0.q
new file mode 100644
index 0000000000000000000000000000000000000000..39fba7d1ac32dab308f590c8282b7d1810d3c4d3
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_0.q
@@ -0,0 +1,27 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT AVG(cbigint),
+       (-(AVG(cbigint))),
+       (-6432 + AVG(cbigint)),
+       STDDEV_POP(cbigint),
+       (-((-6432 + AVG(cbigint)))),
+       ((-((-6432 + AVG(cbigint)))) + (-6432 + AVG(cbigint))),
+       VAR_SAMP(cbigint),
+       (-((-6432 + AVG(cbigint)))),
+       (-6432 + (-((-6432 + AVG(cbigint))))),
+       (-((-6432 + AVG(cbigint)))),
+       ((-((-6432 + AVG(cbigint)))) / (-((-6432 + AVG(cbigint))))),
+       COUNT(*),
+       SUM(cfloat),
+       (VAR_SAMP(cbigint) % STDDEV_POP(cbigint)),
+       (-(VAR_SAMP(cbigint))),
+       ((-((-6432 + AVG(cbigint)))) * (-(AVG(cbigint)))),
+       MIN(ctinyint),
+       (-(MIN(ctinyint)))
+FROM   alltypesorc
+WHERE  (((cstring2 LIKE '%b%')
+         OR ((79.553 != cint)
+             OR (cbigint < cdouble)))
+        OR ((ctinyint >= csmallint)
+            AND ((cboolean2 = 1)
+                 AND (3569 = ctinyint))));
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_1.q
new file mode 100644
index 0000000000000000000000000000000000000000..745aa06f2f5245f1c1b8ef519987c523c6b94fd2
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_1.q
@@ -0,0 +1,21 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT VAR_POP(ctinyint),
+       (VAR_POP(ctinyint) / -26.28),
+       SUM(cfloat),
+       (-1.389 + SUM(cfloat)),
+       (SUM(cfloat) * (-1.389 + SUM(cfloat))),
+       MAX(ctinyint),
+       (-((SUM(cfloat) * (-1.389 + SUM(cfloat))))),
+       MAX(cint),
+       (MAX(cint) * 79.553),
+       VAR_SAMP(cdouble),
+       (10.175 % (-((SUM(cfloat) * (-1.389 + SUM(cfloat)))))),
+       COUNT(cint),
+       (-563 % MAX(cint))
+FROM   alltypesorc
+WHERE  (((cdouble > ctinyint)
+         AND (cboolean2 > 0))
+        OR ((cbigint < ctinyint)
+            OR ((cint > cbigint)
+                OR (cboolean1 < 0))));
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_10.q
new file mode 100644
index 0000000000000000000000000000000000000000..720f38d02b072bc4deda643cec70ef25ced88a93
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_10.q
@@ -0,0 +1,24 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT cdouble,
+       ctimestamp1,
+       ctinyint,
+       cboolean1,
+       cstring1,
+       (-(cdouble)),
+       (cdouble + csmallint),
+       ((cdouble + csmallint) % 33),
+       (-(cdouble)),
+       (ctinyint % cdouble),
+       (ctinyint % csmallint),
+       (-(cdouble)),
+       (cbigint * (ctinyint % csmallint)),
+       (9763215.5639 - (cdouble + csmallint)),
+       (-((-(cdouble))))
+FROM   alltypesorc
+WHERE  (((cstring2 <= '10')
+         OR ((ctinyint > cdouble)
+             AND (-5638.15 >= ctinyint)))
+        OR ((cdouble > 6981)
+            AND ((csmallint = 9763215.5639)
+                 OR (cstring1 LIKE '%a'))));
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_11.q
new file mode 100644
index 0000000000000000000000000000000000000000..329ed28948434054c8bb4c1b1698b110a72e0866
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_11.q
@@ -0,0 +1,15 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT cstring1,
+       cboolean1,
+       cdouble,
+       ctimestamp1,
+       (-3728 * csmallint),
+       (cdouble - 9763215.5639),
+       (-(cdouble)),
+       ((-(cdouble)) + 6981),
+       (cdouble * -5638.15)
+FROM   alltypesorc
+WHERE  ((cstring2 = cstring1)
+        OR ((ctimestamp1 IS NULL)
+            AND (cstring1 LIKE '%a')));
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_12.q
new file mode 100644
index 0000000000000000000000000000000000000000..fef62fc7a0eb65283d8b43dfc36b2d65e174c435
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_12.q
@@ -0,0 +1,32 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT   cbigint,
+         cboolean1,
+         cstring1,
+         ctimestamp1,
+         cdouble,
+         (-6432 * cdouble),
+         (-(cbigint)),
+         COUNT(cbigint),
+         (cbigint * COUNT(cbigint)),
+         STDDEV_SAMP(cbigint),
+         ((-6432 * cdouble) / -6432),
+         (-(((-6432 * cdouble) / -6432))),
+         AVG(cdouble),
+         (-((-6432 * cdouble))),
+         (-5638.15 + cbigint),
+         SUM(cbigint),
+         (AVG(cdouble) / (-6432 * cdouble)),
+         AVG(cdouble),
+         (-((-(((-6432 * cdouble) / -6432))))),
+         (((-6432 * cdouble) / -6432) + (-((-6432 * cdouble)))),
+         STDDEV_POP(cdouble)
+FROM     alltypesorc
+WHERE    (((ctimestamp1 IS NULL)
+           AND ((cboolean1 >= cboolean2)
+                OR (ctinyint != csmallint)))
+          AND ((cstring1 LIKE '%a')
+              OR ((cboolean2 <= 1)
+                  AND (cbigint >= csmallint))))
+GROUP BY cbigint, cboolean1, cstring1, ctimestamp1, cdouble
+ORDER BY ctimestamp1, cdouble, cbigint, cstring1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_13.q
new file mode 100644
index 0000000000000000000000000000000000000000..fad2585d227250477e824afa42c68ed7924a9eea
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_13.q
@@ -0,0 +1,31 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT   cboolean1,
+         ctinyint,
+         ctimestamp1,
+         cfloat,
+         cstring1,
+         (-(ctinyint)),
+         MAX(ctinyint),
+         ((-(ctinyint)) + MAX(ctinyint)),
+         SUM(cfloat),
+         (SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))),
+         (-(SUM(cfloat))),
+         (79.553 * cfloat),
+         STDDEV_POP(cfloat),
+         (-(SUM(cfloat))),
+         STDDEV_POP(ctinyint),
+         (((-(ctinyint)) + MAX(ctinyint)) - 10.175),
+         (-((-(SUM(cfloat))))),
+         (-26.28 / (-((-(SUM(cfloat)))))),
+         MAX(cfloat),
+         ((SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))) / ctinyint),
+         MIN(ctinyint)
+FROM     alltypesorc
+WHERE    (((cfloat < 3569)
+           AND ((10.175 >= cdouble)
+                AND (cboolean1 != 1)))
+          OR ((ctimestamp1 > -29071)
+              AND ((ctimestamp2 != -29071)
+                   AND (ctinyint < 9763215.5639))))
+GROUP BY cboolean1, ctinyint, ctimestamp1, cfloat, cstring1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_14.q
new file mode 100644
index 0000000000000000000000000000000000000000..a121c64edc7b437a6c8a8d9bbce5089b525161ab
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_14.q
@@ -0,0 +1,33 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT   ctimestamp1,
+         cfloat,
+         cstring1,
+         cboolean1,
+         cdouble,
+         (-26.28 + cdouble),
+         (-((-26.28 + cdouble))),
+         STDDEV_SAMP((-((-26.28 + cdouble)))),
+         (cfloat * -26.28),
+         MAX(cfloat),
+         (-(cfloat)),
+         (-(MAX(cfloat))),
+         ((-((-26.28 + cdouble))) / 10.175),
+         STDDEV_POP(cfloat),
+         COUNT(cfloat),
+         (-(((-((-26.28 + cdouble))) / 10.175))),
+         (-1.389 % STDDEV_SAMP((-((-26.28 + cdouble))))),
+         (cfloat - cdouble),
+         VAR_POP(cfloat),
+         (VAR_POP(cfloat) % 10.175),
+         VAR_SAMP(cfloat),
+         (-((cfloat - cdouble)))
+FROM     alltypesorc
+WHERE    (((ctinyint <= cbigint)
+           AND ((cint <= cdouble)
+                OR (ctimestamp2 < ctimestamp1)))
+          AND ((cdouble < ctinyint)
+              AND ((cbigint > -257)
+                  OR (cfloat < cint))))
+GROUP BY ctimestamp1, cfloat, cstring1, cboolean1, cdouble
+ORDER BY cstring1, cfloat, cdouble, ctimestamp1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_15.q
new file mode 100644
index 0000000000000000000000000000000000000000..7daad0b2c50a5c08c5dafeeb3d4aa5777c063389
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_15.q
@@ -0,0 +1,31 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT   cfloat,
+         cboolean1,
+         cdouble,
+         cstring1,
+         ctinyint,
+         cint,
+         ctimestamp1,
+         STDDEV_SAMP(cfloat),
+         (-26.28 - cint),
+         MIN(cdouble),
+         (cdouble * 79.553),
+         (33 % cfloat),
+         STDDEV_SAMP(ctinyint),
+         VAR_POP(ctinyint),
+         (-23 % cdouble),
+         (-(ctinyint)),
+         VAR_SAMP(cint),
+         (cint - cfloat),
+         (-23 % ctinyint),
+         (-((-26.28 - cint))),
+         STDDEV_POP(cint)
+FROM     alltypesorc
+WHERE    (((cstring2 LIKE '%ss%')
+           OR (cstring1 LIKE '10%'))
+          OR ((cint >= -75)
+              AND ((ctinyint = csmallint)
+                   AND (cdouble >= -3728))))
+GROUP BY cfloat, cboolean1, cdouble, cstring1, ctinyint, cint, ctimestamp1
+ORDER BY cfloat, cboolean1, cdouble, cstring1, ctinyint, cint, ctimestamp1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_16.q
new file mode 100644
index 0000000000000000000000000000000000000000..39a94026804960c19e4e6ae6398d4ec33ae7a4e3
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_16.q
@@ -0,0 +1,20 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT   cstring1,
+         cdouble,
+         ctimestamp1,
+         (cdouble - 9763215.5639),
+         (-((cdouble - 9763215.5639))),
+         COUNT(cdouble),
+         STDDEV_SAMP(cdouble),
+         (-(STDDEV_SAMP(cdouble))),
+         (STDDEV_SAMP(cdouble) * COUNT(cdouble)),
+         MIN(cdouble),
+         (9763215.5639 / cdouble),
+         (COUNT(cdouble) / -1.389),
+         STDDEV_SAMP(cdouble)
+FROM     alltypesorc
+WHERE    ((cstring2 LIKE '%b%')
+          AND ((cdouble >= -1.389)
+              OR (cstring1 < 'a')))
+GROUP BY cstring1, cdouble, ctimestamp1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_2.q
new file mode 100644
index 0000000000000000000000000000000000000000..b8647a434414b2f70e471047f8fed552dbb7ceee
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_2.q
@@ -0,0 +1,23 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT AVG(csmallint),
+       (AVG(csmallint) % -563),
+       (AVG(csmallint) + 762),
+       SUM(cfloat),
+       VAR_POP(cbigint),
+       (-(VAR_POP(cbigint))),
+       (SUM(cfloat) - AVG(csmallint)),
+       COUNT(*),
+       (-((SUM(cfloat) - AVG(csmallint)))),
+       (VAR_POP(cbigint) - 762),
+       MIN(ctinyint),
+       ((-(VAR_POP(cbigint))) + MIN(ctinyint)),
+       AVG(cdouble),
+       (((-(VAR_POP(cbigint))) + MIN(ctinyint)) - SUM(cfloat))
+FROM   alltypesorc
+WHERE  (((ctimestamp1 < ctimestamp2)
+         AND ((cstring2 LIKE 'b%')
+              AND (cfloat <= -5638.15)))
+        OR ((cdouble < ctinyint)
+            AND ((-10669 != ctimestamp2)
+                 OR (359 > cint))));
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_3.q
new file mode 100644
index 0000000000000000000000000000000000000000..1d53994519db3624a243e04528714182366fe758
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_3.q
@@ -0,0 +1,25 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT STDDEV_SAMP(csmallint),
+       (STDDEV_SAMP(csmallint) - 10.175),
+       STDDEV_POP(ctinyint),
+       (STDDEV_SAMP(csmallint) * (STDDEV_SAMP(csmallint) - 10.175)),
+       (-(STDDEV_POP(ctinyint))),
+       (STDDEV_SAMP(csmallint) % 79.553),
+       (-((STDDEV_SAMP(csmallint) * (STDDEV_SAMP(csmallint) - 10.175)))),
+       STDDEV_SAMP(cfloat),
+       (-(STDDEV_SAMP(csmallint))),
+       SUM(cfloat),
+       ((-((STDDEV_SAMP(csmallint) * (STDDEV_SAMP(csmallint) - 10.175)))) / (STDDEV_SAMP(csmallint) - 10.175)),
+       (-((STDDEV_SAMP(csmallint) - 10.175))),
+       AVG(cint),
+       (-3728 - STDDEV_SAMP(csmallint)),
+       STDDEV_POP(cint),
+       (AVG(cint) / STDDEV_SAMP(cfloat))
+FROM   alltypesorc
+WHERE  (((cint <= cfloat)
+         AND ((79.553 != cbigint)
+              AND (ctimestamp2 = -29071)))
+        OR ((cbigint > cdouble)
+            AND ((79.553 <= csmallint)
+                 AND (ctimestamp1 > ctimestamp2))));
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_4.q
new file mode 100644
index 0000000000000000000000000000000000000000..1eb324d1902e875fcb1e67ffceb28f769141746f
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_4.q
@@ -0,0 +1,23 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT SUM(cint),
+       (SUM(cint) * -563),
+       (-3728 + SUM(cint)),
+       STDDEV_POP(cdouble),
+       (-(STDDEV_POP(cdouble))),
+       AVG(cdouble),
+       ((SUM(cint) * -563) % SUM(cint)),
+       (((SUM(cint) * -563) % SUM(cint)) / AVG(cdouble)),
+       VAR_POP(cdouble),
+       (-((((SUM(cint) * -563) % SUM(cint)) / AVG(cdouble)))),
+       ((-3728 + SUM(cint)) - (SUM(cint) * -563)),
+       MIN(ctinyint),
+       MIN(ctinyint),
+       (MIN(ctinyint) * (-((((SUM(cint) * -563) % SUM(cint)) / AVG(cdouble)))))
+FROM   alltypesorc
+WHERE  (((csmallint >= cint)
+         OR ((-89010 >= ctinyint)
+             AND (cdouble > 79.553)))
+        OR ((-563 != cbigint)
+            AND ((ctinyint != cbigint)
+                 OR (-3728 >= cdouble))));
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_5.q
new file mode 100644
index 0000000000000000000000000000000000000000..826b20fd4f124f3226334594b842bad906b6966a
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_5.q
@@ -0,0 +1,20 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT MAX(csmallint),
+       (MAX(csmallint) * -75),
+       COUNT(*),
+       ((MAX(csmallint) * -75) / COUNT(*)),
+       (6981 * MAX(csmallint)),
+       MIN(csmallint),
+       (-(MIN(csmallint))),
+       (197 % ((MAX(csmallint) * -75) / COUNT(*))),
+       SUM(cint),
+       MAX(ctinyint),
+       (-(MAX(ctinyint))),
+       ((-(MAX(ctinyint))) + MAX(ctinyint))
+FROM   alltypesorc
+WHERE  (((cboolean2 IS NOT NULL)
+         AND (cstring1 LIKE '%b%'))
+        OR ((ctinyint = cdouble)
+            AND ((ctimestamp2 IS NOT NULL)
+                 AND (cstring2 LIKE 'a'))));
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_6.q
new file mode 100644
index 0000000000000000000000000000000000000000..2b59f10ed89c3e71e4e195499035149d72fddb9f
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_6.q
@@ -0,0 +1,21 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT cboolean1,
+       cfloat,
+       cstring1,
+       (988888 * csmallint),
+       (-(csmallint)),
+       (-(cfloat)),
+       (-26.28 / cfloat),
+       (cfloat * 359),
+       (cint % ctinyint),
+       (-(cdouble)),
+       (ctinyint - -75),
+       (762 * (cint % ctinyint))
+FROM   alltypesorc
+WHERE  ((ctinyint != 0)
+        AND ((((cboolean1 <= 0)
+          AND (cboolean2 >= cboolean1))
+          OR ((cbigint IS NOT NULL)
+              AND ((cstring2 LIKE '%a')
+                   OR (cfloat <= -257))))));
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_7.q
new file mode 100644
index 0000000000000000000000000000000000000000..20c1148659195649769890c931abd0ed8cef53c1
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_7.q
@@ -0,0 +1,25 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT cboolean1,
+       cbigint,
+       csmallint,
+       ctinyint,
+       ctimestamp1,
+       cstring1,
+       (cbigint + cbigint),
+       (csmallint % -257),
+       (-(csmallint)),
+       (-(ctinyint)),
+       ((-(ctinyint)) + 17),
+       (cbigint * (-(csmallint))),
+       (cint % csmallint),
+       (-(ctinyint)),
+       ((-(ctinyint)) % ctinyint)
+FROM   alltypesorc
+WHERE  ((ctinyint != 0)
+        AND (((ctimestamp1 <= 0)
+          OR ((ctinyint = cint)
+               OR (cstring2 LIKE 'ss')))
+          AND ((988888 < cdouble)
+              OR ((ctimestamp2 > -29071)
+                  AND (3569 >= cdouble)))));
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_8.q
new file mode 100644
index 0000000000000000000000000000000000000000..98b3385f165a9e4167712fb99565aa86c59e025c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_8.q
@@ -0,0 +1,23 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT ctimestamp1,
+       cdouble,
+       cboolean1,
+       cstring1,
+       cfloat,
+       (-(cdouble)),
+       (-5638.15 - cdouble),
+       (cdouble * -257),
+       (cint + cfloat),
+       ((-(cdouble)) + cbigint),
+       (-(cdouble)),
+       (-1.389 - cfloat),
+       (-(cfloat)),
+       ((-5638.15 - cdouble) + (cint + cfloat))
+FROM   alltypesorc
+WHERE  (((cstring2 IS NOT NULL)
+         AND ((ctimestamp1 <= -29071)
+             AND (ctimestamp2 != 16558)))
+        OR ((cfloat < -6432)
+            OR ((cboolean1 IS NOT NULL)
+                AND (cdouble = 988888))));
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_9.q
new file mode 100644
index 0000000000000000000000000000000000000000..252c426d64a5e181322a12871c1b5dacf48aedf1
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_9.q
@@ -0,0 +1,24 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT   cfloat,
+         cstring1,
+         cint,
+         ctimestamp1,
+         cdouble,
+         cbigint,
+         (cfloat / ctinyint),
+         (cint % cbigint),
+         (-(cdouble)),
+         (cdouble + (cfloat / ctinyint)),
+         (cdouble / cint),
+         (-((-(cdouble)))),
+         (9763215.5639 % cbigint),
+         (2563.58 + (-((-(cdouble)))))
+FROM     alltypesorc
+WHERE    (((cbigint > -23)
+           AND ((cdouble != 988888)
+                OR (cint > -863.257)))
+          AND ((ctinyint >= 33)
+              OR ((csmallint >= cbigint)
+                  OR (cfloat = cdouble))))
+ORDER BY cbigint, cfloat;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_decimal_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_decimal_date.q
new file mode 100644
index 0000000000000000000000000000000000000000..2b82a5aa0452a52367c8e268d3b6d1443c0a470f
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_decimal_date.q
@@ -0,0 +1,4 @@
+CREATE TABLE date_decimal_test STORED AS ORC AS SELECT cint, cdouble, CAST (CAST (cint AS TIMESTAMP) AS DATE) AS cdate, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal FROM alltypesorc;
+SET hive.vectorized.execution.enabled=true;
+EXPLAIN SELECT cdate, cdecimal from date_decimal_test where cint IS NOT NULL AND cdouble IS NOT NULL LIMIT 10;
+SELECT cdate, cdecimal from date_decimal_test where cint IS NOT NULL AND cdouble IS NOT NULL LIMIT 10;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_div0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_div0.q
new file mode 100644
index 0000000000000000000000000000000000000000..69e388a28e411fe50e882888068b8b461339b5a5
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_div0.q
@@ -0,0 +1,24 @@
+SET hive.vectorized.execution.enabled = true;
+
+-- TODO: add more stuff here after HIVE-5918 is fixed, such as cbigint and constants
+explain
+select cdouble / 0.0 from alltypesorc limit 100;
+select cdouble / 0.0 from alltypesorc limit 100;
+
+-- There are no zeros in the table, but there is 988888, so use it as zero
+
+-- TODO: add more stuff here after HIVE-5918 is fixed, such as cbigint and constants as numerators
+explain
+select (cbigint - 988888L) as s1, cdouble / (cbigint - 988888L) as s2, 1.2 / (cbigint - 988888L) 
+from alltypesorc where cbigint > 0 and cbigint < 100000000 order by s1, s2 limit 100;
+select (cbigint - 988888L) as s1, cdouble / (cbigint - 988888L) as s2, 1.2 / (cbigint - 988888L) 
+from alltypesorc where cbigint > 0 and cbigint < 100000000 order by s1, s2 limit 100;
+
+-- There are no zeros in the table, but there is -200.0, so use it as zero
+
+explain
+select (cdouble + 200.0) as s1, cbigint / (cdouble + 200.0) as s2, (cdouble + 200.0) / (cdouble + 200.0), cbigint / (cdouble + 200.0), 1 / (cdouble + 200.0), 1.2 / (cdouble + 200.0) 
+from alltypesorc where cdouble >= -500 and cdouble < -199 order by s1, s2 limit 100;
+select (cdouble + 200.0) as s1, cbigint / (cdouble + 200.0) as s2, (cdouble + 200.0) / (cdouble + 200.0), cbigint / (cdouble + 200.0), 1 / (cdouble + 200.0), 1.2 / (cdouble + 200.0) 
+from alltypesorc where cdouble >= -500 and cdouble < -199 order by s1, s2 limit 100;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_limit.q
new file mode 100644
index 0000000000000000000000000000000000000000..094a8d26a3842385374683a75a3dbdacf6775637
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_limit.q
@@ -0,0 +1,37 @@
+SET hive.vectorized.execution.enabled=true;
+explain SELECT cbigint, cdouble FROM alltypesorc WHERE cbigint < cdouble and cint > 0 limit 7;
+SELECT cbigint, cdouble FROM alltypesorc WHERE cbigint < cdouble and cint > 0 limit 7;
+
+set hive.optimize.reducededuplication.min.reducer=1;
+set hive.limit.pushdown.memory.usage=0.3f;
+
+-- HIVE-3562 Some limit can be pushed down to map stage - c/p parts from limit_pushdown
+
+explain
+select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20;
+select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20;
+
+-- deduped RS
+explain
+select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20;
+select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20;
+
+-- distincts
+explain
+select distinct(ctinyint) from alltypesorc limit 20;
+select distinct(ctinyint) from alltypesorc limit 20;
+
+explain
+select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint limit 20;
+select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint limit 20;
+
+-- limit zero
+explain
+select ctinyint,cdouble from alltypesorc order by ctinyint limit 0;
+select ctinyint,cdouble from alltypesorc order by ctinyint limit 0;
+
+-- 2MR (applied to last RS)
+explain
+select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20;
+select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_nested_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_nested_udf.q
new file mode 100644
index 0000000000000000000000000000000000000000..bb50f9b853287dad4d6f82ecd871a213ea039c8c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_nested_udf.q
@@ -0,0 +1,3 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT SUM(abs(ctinyint)) from alltypesorc;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_not.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_not.q
new file mode 100644
index 0000000000000000000000000000000000000000..bfd3dd722190c0e9c61e72a0752583b81851061f
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_not.q
@@ -0,0 +1,27 @@
+SET hive.vectorized.execution.enabled=true;
+SELECT AVG(cbigint),
+       (-(AVG(cbigint))),
+       (-6432 + AVG(cbigint)),
+       STDDEV_POP(cbigint),
+       (-((-6432 + AVG(cbigint)))),
+       ((-((-6432 + AVG(cbigint)))) + (-6432 + AVG(cbigint))),
+       VAR_SAMP(cbigint),
+       (-((-6432 + AVG(cbigint)))),
+       (-6432 + (-((-6432 + AVG(cbigint))))),
+       (-((-6432 + AVG(cbigint)))),
+       ((-((-6432 + AVG(cbigint)))) / (-((-6432 + AVG(cbigint))))),
+       COUNT(*),
+       SUM(cfloat),
+       (VAR_SAMP(cbigint) % STDDEV_POP(cbigint)),
+       (-(VAR_SAMP(cbigint))),
+       ((-((-6432 + AVG(cbigint)))) * (-(AVG(cbigint)))),
+       MIN(ctinyint),
+       (-(MIN(ctinyint)))
+FROM   alltypesorc
+WHERE  (((cstring2 LIKE '%b%')
+         OR ((79.553 != cint)
+             OR (NOT(cbigint >= cdouble))))
+        OR ((ctinyint >= csmallint)
+            AND (NOT ((cboolean2 != 1)
+                 OR (3569 != ctinyint)))));
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part.q
new file mode 100644
index 0000000000000000000000000000000000000000..0e34585b5a8ba0c32055a6ccabab2a9d8a55bc1c
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part.q
@@ -0,0 +1,7 @@
+SET hive.vectorized.execution.enabled=true;
+CREATE TABLE alltypesorc_part(ctinyint tinyint, csmallint smallint, cint int, cbigint bigint, cfloat float, cdouble double, cstring1 string, cstring2 string, ctimestamp1 timestamp, ctimestamp2 timestamp, cboolean1 boolean, cboolean2 boolean) partitioned by (ds string) STORED AS ORC;
+insert overwrite table alltypesorc_part partition (ds='2011') select * from alltypesorc limit 100;
+insert overwrite table alltypesorc_part partition (ds='2012') select * from alltypesorc limit 100;
+
+select count(cdouble), cint from alltypesorc_part where ds='2011' group by cint limit 10;
+select count(*) from alltypesorc_part A join alltypesorc_part B on A.ds=B.ds;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part_project.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part_project.q
new file mode 100644
index 0000000000000000000000000000000000000000..c68ce56fd47dbe0d0e3b25ae3031337e9549a1af
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part_project.q
@@ -0,0 +1,7 @@
+SET hive.vectorized.execution.enabled=true;
+CREATE TABLE alltypesorc_part(ctinyint tinyint, csmallint smallint, cint int, cbigint bigint, cfloat float, cdouble double, cstring1 string, cstring2 string, ctimestamp1 timestamp, ctimestamp2 timestamp, cboolean1 boolean, cboolean2 boolean) partitioned by (ds string) STORED AS ORC;
+insert overwrite table alltypesorc_part partition (ds='2011') select * from alltypesorc limit 100;
+insert overwrite table alltypesorc_part partition (ds='2012') select * from alltypesorc limit 100;
+
+explain select (cdouble+2) c1 from alltypesorc_part order by c1 limit 10;
+select (cdouble+2) c1 from alltypesorc_part order by c1 limit 10;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_pushdown.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_pushdown.q
new file mode 100644
index 0000000000000000000000000000000000000000..bafe5504789e89da51659cfcb44a159b794865e4
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_pushdown.q
@@ -0,0 +1,4 @@
+SET hive.vectorized.execution.enabled=true;
+SET hive.optimize.index.filter=true;
+explain SELECT AVG(cbigint) FROM alltypesorc WHERE cbigint < cdouble;
+SELECT AVG(cbigint) FROM alltypesorc WHERE cbigint < cdouble;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_short_regress.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_short_regress.q
new file mode 100644
index 0000000000000000000000000000000000000000..638a31ff413b74adca94dbf9872b850a29270fb9
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_short_regress.q
@@ -0,0 +1,852 @@
+SET hive.vectorized.execution.enabled=true;
+
+-- If you look at ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/OrcFileGenerator.java
+-- which is the data generation class you'll see that those values are specified in the
+-- initializeFixedPointValues for each data type. When I created the queries I usedthose values
+-- where I needed scalar values to ensure that when the queries executed their predicates would be
+-- filtering on values that are guaranteed to exist.
+
+-- Beyond those values, all the other data in the alltypesorc file is random, but there is a
+-- specific pattern to the data that is important for coverage. In orc and subsequently
+-- vectorization there are a number of optimizations for certain data patterns: AllValues, NoNulls,
+-- RepeatingValue, RepeatingNull. The data in alltypesorc is generated such that each column has
+-- exactly 3 batches of each data pattern. This gives us coverage for the vector expression
+-- optimizations and ensure the metadata in appropriately set on the row batch object which are
+-- reused across batches. 
+
+-- For the queries themselves in order to efficiently cover as much of the new vectorization
+-- functionality as I could I used a number of different techniques to create the
+-- vectorization_short_regress.q test suite, primarily equivalence classes, and pairwise
+-- combinations.
+
+-- First I divided the search space into a number of dimensions such as type, aggregate function,
+-- filter operation, arithmetic operation, etc. The types were explored as equivalence classes of
+-- long, double, time, string, and bool. Also, rather than creating a very large number of small
+-- queries the resulting vectors were grouped by compatible dimensions to reduce the number of
+-- queries.
+
+-- TargetTypeClasses: Long, Timestamp, Double, String, Bool
+-- Functions: Avg, Sum, StDevP, StDev, Var, Min, Count
+-- ArithmeticOps: Add, Multiply, Subtract, Divide
+-- FilterOps: Equal, NotEqual, GreaterThan, LessThan, LessThanOrEqual
+-- GroupBy: NoGroupByProjectAggs
+EXPLAIN SELECT AVG(cint),
+       (AVG(cint) + -3728),
+       (-((AVG(cint) + -3728))),
+       (-((-((AVG(cint) + -3728))))),
+       ((-((-((AVG(cint) + -3728))))) * (AVG(cint) + -3728)),
+       SUM(cdouble),
+       (-(AVG(cint))),
+       STDDEV_POP(cint),
+       (((-((-((AVG(cint) + -3728))))) * (AVG(cint) + -3728)) * (-((-((AVG(cint) + -3728)))))),
+       STDDEV_SAMP(csmallint),
+       (-(STDDEV_POP(cint))),
+       (STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)))))),
+       ((STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)))))) * STDDEV_POP(cint)),
+       VAR_SAMP(cint),
+       AVG(cfloat),
+       (10.175 - VAR_SAMP(cint)),
+       (-((10.175 - VAR_SAMP(cint)))),
+       ((-(STDDEV_POP(cint))) / -563),
+       STDDEV_SAMP(cint),
+       (-(((-(STDDEV_POP(cint))) / -563))),
+       (AVG(cint) / SUM(cdouble)),
+       MIN(ctinyint),
+       COUNT(csmallint),
+       (MIN(ctinyint) / ((-(STDDEV_POP(cint))) / -563)),
+       (-((AVG(cint) / SUM(cdouble))))
+FROM   alltypesorc
+WHERE  ((762 = cbigint)
+        OR ((csmallint < cfloat)
+            AND ((ctimestamp2 > -10669)
+                 AND (cdouble != cint)))
+        OR (cstring1 = 'a')
+           OR ((cbigint <= -1.389)
+               AND ((cstring2 != 'a')
+                    AND ((79.553 != cint)
+                         AND (cboolean2 != cboolean1)))));
+SELECT AVG(cint),
+       (AVG(cint) + -3728),
+       (-((AVG(cint) + -3728))),
+       (-((-((AVG(cint) + -3728))))),
+       ((-((-((AVG(cint) + -3728))))) * (AVG(cint) + -3728)),
+       SUM(cdouble),
+       (-(AVG(cint))),
+       STDDEV_POP(cint),
+       (((-((-((AVG(cint) + -3728))))) * (AVG(cint) + -3728)) * (-((-((AVG(cint) + -3728)))))),
+       STDDEV_SAMP(csmallint),
+       (-(STDDEV_POP(cint))),
+       (STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)))))),
+       ((STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)))))) * STDDEV_POP(cint)),
+       VAR_SAMP(cint),
+       AVG(cfloat),
+       (10.175 - VAR_SAMP(cint)),
+       (-((10.175 - VAR_SAMP(cint)))),
+       ((-(STDDEV_POP(cint))) / -563),
+       STDDEV_SAMP(cint),
+       (-(((-(STDDEV_POP(cint))) / -563))),
+       (AVG(cint) / SUM(cdouble)),
+       MIN(ctinyint),
+       COUNT(csmallint),
+       (MIN(ctinyint) / ((-(STDDEV_POP(cint))) / -563)),
+       (-((AVG(cint) / SUM(cdouble))))
+FROM   alltypesorc
+WHERE  ((762 = cbigint)
+        OR ((csmallint < cfloat)
+            AND ((ctimestamp2 > -10669)
+                 AND (cdouble != cint)))
+        OR (cstring1 = 'a')
+           OR ((cbigint <= -1.389)
+               AND ((cstring2 != 'a')
+                    AND ((79.553 != cint)
+                         AND (cboolean2 != cboolean1)))));
+
+-- TargetTypeClasses: Long, Bool, Double, String, Timestamp
+-- Functions: Max, VarP, StDevP, Avg, Min, StDev, Var
+-- ArithmeticOps: Divide, Multiply, Remainder, Subtract
+-- FilterOps: LessThan, LessThanOrEqual, GreaterThan, GreaterThanOrEqual, Like, RLike
+-- GroupBy: NoGroupByProjectAggs
+EXPLAIN SELECT MAX(cint),
+       (MAX(cint) / -3728),
+       (MAX(cint) * -3728),
+       VAR_POP(cbigint),
+       (-((MAX(cint) * -3728))),
+       STDDEV_POP(csmallint),
+       (-563 % (MAX(cint) * -3728)),
+       (VAR_POP(cbigint) / STDDEV_POP(csmallint)),
+       (-(STDDEV_POP(csmallint))),
+       MAX(cdouble),
+       AVG(ctinyint),
+       (STDDEV_POP(csmallint) - 10.175),
+       MIN(cint),
+       ((MAX(cint) * -3728) % (STDDEV_POP(csmallint) - 10.175)),
+       (-(MAX(cdouble))),
+       MIN(cdouble),
+       (MAX(cdouble) % -26.28),
+       STDDEV_SAMP(csmallint),
+       (-((MAX(cint) / -3728))),
+       ((-((MAX(cint) * -3728))) % (-563 % (MAX(cint) * -3728))),
+       ((MAX(cint) / -3728) - AVG(ctinyint)),
+       (-((MAX(cint) * -3728))),
+       VAR_SAMP(cint)
+FROM   alltypesorc
+WHERE  (((cbigint <= 197)
+         AND (cint < cbigint))
+        OR ((cdouble >= -26.28)
+            AND (csmallint > cdouble))
+        OR ((ctinyint > cfloat)
+            AND (cstring1 RLIKE '.*ss.*'))
+           OR ((cfloat > 79.553)
+               AND (cstring2 LIKE '10%')));
+SELECT MAX(cint),
+       (MAX(cint) / -3728),
+       (MAX(cint) * -3728),
+       VAR_POP(cbigint),
+       (-((MAX(cint) * -3728))),
+       STDDEV_POP(csmallint),
+       (-563 % (MAX(cint) * -3728)),
+       (VAR_POP(cbigint) / STDDEV_POP(csmallint)),
+       (-(STDDEV_POP(csmallint))),
+       MAX(cdouble),
+       AVG(ctinyint),
+       (STDDEV_POP(csmallint) - 10.175),
+       MIN(cint),
+       ((MAX(cint) * -3728) % (STDDEV_POP(csmallint) - 10.175)),
+       (-(MAX(cdouble))),
+       MIN(cdouble),
+       (MAX(cdouble) % -26.28),
+       STDDEV_SAMP(csmallint),
+       (-((MAX(cint) / -3728))),
+       ((-((MAX(cint) * -3728))) % (-563 % (MAX(cint) * -3728))),
+       ((MAX(cint) / -3728) - AVG(ctinyint)),
+       (-((MAX(cint) * -3728))),
+       VAR_SAMP(cint)
+FROM   alltypesorc
+WHERE  (((cbigint <= 197)
+         AND (cint < cbigint))
+        OR ((cdouble >= -26.28)
+            AND (csmallint > cdouble))
+        OR ((ctinyint > cfloat)
+            AND (cstring1 RLIKE '.*ss.*'))
+           OR ((cfloat > 79.553)
+               AND (cstring2 LIKE '10%')));
+
+-- TargetTypeClasses: String, Long, Bool, Double, Timestamp
+-- Functions: VarP, Count, Max, StDevP, StDev, Avg
+-- ArithmeticOps: Subtract, Remainder, Multiply, Add
+-- FilterOps: Equal, LessThanOrEqual, GreaterThan, Like, LessThan
+-- GroupBy: NoGroupByProjectAggs
+EXPLAIN SELECT VAR_POP(cbigint),
+       (-(VAR_POP(cbigint))),
+       (VAR_POP(cbigint) - (-(VAR_POP(cbigint)))),
+       COUNT(*),
+       (COUNT(*) % 79.553),
+       MAX(ctinyint),
+       (COUNT(*) - (-(VAR_POP(cbigint)))),
+       (-((-(VAR_POP(cbigint))))),
+       (-1 % (-(VAR_POP(cbigint)))),
+       COUNT(*),
+       (-(COUNT(*))),
+       STDDEV_POP(csmallint),
+       (-((-((-(VAR_POP(cbigint))))))),
+       (762 * (-(COUNT(*)))),
+       MAX(cint),
+       (MAX(ctinyint) + (762 * (-(COUNT(*))))),
+       ((-(VAR_POP(cbigint))) + MAX(cint)),
+       STDDEV_SAMP(cdouble),
+       ((-(COUNT(*))) % COUNT(*)),
+       COUNT(ctinyint),
+       AVG(ctinyint),
+       (-3728 % (MAX(ctinyint) + (762 * (-(COUNT(*))))))
+FROM   alltypesorc
+WHERE  ((ctimestamp1 = ctimestamp2)
+        OR (762 = cfloat)
+        OR (cstring1 = 'ss')
+           OR ((csmallint <= cbigint)
+               AND (1 = cboolean2))
+              OR ((cboolean1 IS NOT NULL)
+                  AND ((ctimestamp2 IS NOT NULL)
+                       AND (cstring2 > 'a'))));
+SELECT VAR_POP(cbigint),
+       (-(VAR_POP(cbigint))),
+       (VAR_POP(cbigint) - (-(VAR_POP(cbigint)))),
+       COUNT(*),
+       (COUNT(*) % 79.553),
+       MAX(ctinyint),
+       (COUNT(*) - (-(VAR_POP(cbigint)))),
+       (-((-(VAR_POP(cbigint))))),
+       (-1 % (-(VAR_POP(cbigint)))),
+       COUNT(*),
+       (-(COUNT(*))),
+       STDDEV_POP(csmallint),
+       (-((-((-(VAR_POP(cbigint))))))),
+       (762 * (-(COUNT(*)))),
+       MAX(cint),
+       (MAX(ctinyint) + (762 * (-(COUNT(*))))),
+       ((-(VAR_POP(cbigint))) + MAX(cint)),
+       STDDEV_SAMP(cdouble),
+       ((-(COUNT(*))) % COUNT(*)),
+       COUNT(ctinyint),
+       AVG(ctinyint),
+       (-3728 % (MAX(ctinyint) + (762 * (-(COUNT(*))))))
+FROM   alltypesorc
+WHERE  ((ctimestamp1 = ctimestamp2)
+        OR (762 = cfloat)
+        OR (cstring1 = 'ss')
+           OR ((csmallint <= cbigint)
+               AND (1 = cboolean2))
+              OR ((cboolean1 IS NOT NULL)
+                  AND ((ctimestamp2 IS NOT NULL)
+                       AND (cstring2 > 'a'))));
+
+-- TargetTypeClasses: String, Bool, Timestamp, Long, Double
+-- Functions: Avg, Max, StDev, VarP
+-- ArithmeticOps: Add, Divide, Remainder, Multiply
+-- FilterOps: LessThanOrEqual, NotEqual, GreaterThanOrEqual, LessThan, Equal
+-- GroupBy: NoGroupByProjectAggs
+EXPLAIN SELECT AVG(ctinyint),
+       (AVG(ctinyint) + 6981),
+       ((AVG(ctinyint) + 6981) + AVG(ctinyint)),
+       MAX(cbigint),
+       (((AVG(ctinyint) + 6981) + AVG(ctinyint)) / AVG(ctinyint)),
+       (-((AVG(ctinyint) + 6981))),
+       STDDEV_SAMP(cint),
+       (AVG(ctinyint) % (-((AVG(ctinyint) + 6981)))),
+       VAR_POP(cint),
+       VAR_POP(cbigint),
+       (-(MAX(cbigint))),
+       ((-(MAX(cbigint))) / STDDEV_SAMP(cint)),
+       MAX(cfloat),
+       (VAR_POP(cbigint) * -26.28)
+FROM   alltypesorc
+WHERE  (((ctimestamp2 <= ctimestamp1)
+         AND ((cbigint != cdouble)
+              AND ('ss' <= cstring1)))
+        OR ((csmallint < ctinyint)
+            AND (ctimestamp1 >= 0))
+           OR (cfloat = 17));
+SELECT AVG(ctinyint),
+       (AVG(ctinyint) + 6981),
+       ((AVG(ctinyint) + 6981) + AVG(ctinyint)),
+       MAX(cbigint),
+       (((AVG(ctinyint) + 6981) + AVG(ctinyint)) / AVG(ctinyint)),
+       (-((AVG(ctinyint) + 6981))),
+       STDDEV_SAMP(cint),
+       (AVG(ctinyint) % (-((AVG(ctinyint) + 6981)))),
+       VAR_POP(cint),
+       VAR_POP(cbigint),
+       (-(MAX(cbigint))),
+       ((-(MAX(cbigint))) / STDDEV_SAMP(cint)),
+       MAX(cfloat),
+       (VAR_POP(cbigint) * -26.28)
+FROM   alltypesorc
+WHERE  (((ctimestamp2 <= ctimestamp1)
+         AND ((cbigint != cdouble)
+              AND ('ss' <= cstring1)))
+        OR ((csmallint < ctinyint)
+            AND (ctimestamp1 >= 0))
+           OR (cfloat = 17));
+
+-- TargetTypeClasses: Timestamp, String, Long, Double, Bool
+-- Functions: Max, Avg, Min, Var, StDev, Count, StDevP, Sum
+-- ArithmeticOps: Multiply, Subtract, Add, Divide
+-- FilterOps: Like, NotEqual, LessThan, GreaterThanOrEqual, GreaterThan, RLike
+-- GroupBy: NoGroupByProjectColumns
+EXPLAIN SELECT cint,
+       cdouble,
+       ctimestamp2,
+       cstring1,
+       cboolean2,
+       ctinyint,
+       cfloat,
+       ctimestamp1,
+       csmallint,
+       cbigint,
+       (-3728 * cbigint),
+       (-(cint)),
+       (-863.257 - cint),
+       (-(csmallint)),
+       (csmallint - (-(csmallint))),
+       ((csmallint - (-(csmallint))) + (-(csmallint))),
+       (cint / cint),
+       ((-863.257 - cint) - -26.28),
+       (-(cfloat)),
+       (cdouble * -89010),
+       (ctinyint / 988888),
+       (-(ctinyint)),
+       (79.553 / ctinyint)
+FROM   alltypesorc
+WHERE  (((cstring1 RLIKE 'a.*')
+         AND (cstring2 LIKE '%ss%'))
+        OR ((1 != cboolean2)
+            AND ((csmallint < 79.553)
+                 AND (-257 != ctinyint)))
+        OR ((cdouble > ctinyint)
+            AND (cfloat >= cint))
+           OR ((cint < cbigint)
+               AND (ctinyint > cbigint)));
+SELECT cint,
+       cdouble,
+       ctimestamp2,
+       cstring1,
+       cboolean2,
+       ctinyint,
+       cfloat,
+       ctimestamp1,
+       csmallint,
+       cbigint,
+       (-3728 * cbigint),
+       (-(cint)),
+       (-863.257 - cint),
+       (-(csmallint)),
+       (csmallint - (-(csmallint))),
+       ((csmallint - (-(csmallint))) + (-(csmallint))),
+       (cint / cint),
+       ((-863.257 - cint) - -26.28),
+       (-(cfloat)),
+       (cdouble * -89010),
+       (ctinyint / 988888),
+       (-(ctinyint)),
+       (79.553 / ctinyint)
+FROM   alltypesorc
+WHERE  (((cstring1 RLIKE 'a.*')
+         AND (cstring2 LIKE '%ss%'))
+        OR ((1 != cboolean2)
+            AND ((csmallint < 79.553)
+                 AND (-257 != ctinyint)))
+        OR ((cdouble > ctinyint)
+            AND (cfloat >= cint))
+           OR ((cint < cbigint)
+               AND (ctinyint > cbigint)));
+
+-- TargetTypeClasses: Long, String, Double, Bool, Timestamp
+-- Functions: VarP, Var, StDev, StDevP, Max, Sum
+-- ArithmeticOps: Divide, Remainder, Subtract, Multiply
+-- FilterOps: Equal, LessThanOrEqual, LessThan, Like, GreaterThanOrEqual, NotEqual, GreaterThan
+-- GroupBy: NoGroupByProjectColumns
+EXPLAIN SELECT cint,
+       cbigint,
+       cstring1,
+       cboolean1,
+       cfloat,
+       cdouble,
+       ctimestamp2,
+       csmallint,
+       cstring2,
+       cboolean2,
+       (cint / cbigint),
+       (cbigint % 79.553),
+       (-((cint / cbigint))),
+       (10.175 % cfloat),
+       (-(cfloat)),
+       (cfloat - (-(cfloat))),
+       ((cfloat - (-(cfloat))) % -6432),
+       (cdouble * csmallint),
+       (-(cdouble)),
+       (-(cbigint)),
+       (cfloat - (cint / cbigint)),
+       (-(csmallint)),
+       (3569 % cbigint),
+       (359 - cdouble),
+       (-(csmallint))
+FROM   alltypesorc
+WHERE  (((197 > ctinyint)
+         AND (cint = cbigint))
+        OR (cbigint = 359)
+        OR (cboolean1 < 0)
+           OR ((cstring1 LIKE '%ss')
+               AND (cfloat <= ctinyint)));
+
+SELECT cint,
+       cbigint,
+       cstring1,
+       cboolean1,
+       cfloat,
+       cdouble,
+       ctimestamp2,
+       csmallint,
+       cstring2,
+       cboolean2,
+       (cint / cbigint),
+       (cbigint % 79.553),
+       (-((cint / cbigint))),
+       (10.175 % cfloat),
+       (-(cfloat)),
+       (cfloat - (-(cfloat))),
+       ((cfloat - (-(cfloat))) % -6432),
+       (cdouble * csmallint),
+       (-(cdouble)),
+       (-(cbigint)),
+       (cfloat - (cint / cbigint)),
+       (-(csmallint)),
+       (3569 % cbigint),
+       (359 - cdouble),
+       (-(csmallint))
+FROM   alltypesorc
+WHERE  (((197 > ctinyint)
+         AND (cint = cbigint))
+        OR (cbigint = 359)
+        OR (cboolean1 < 0)
+           OR ((cstring1 LIKE '%ss')
+               AND (cfloat <= ctinyint)));
+
+-- TargetTypeClasses: String, Bool, Double, Long, Timestamp
+-- Functions: Sum, Max, Avg, Var, StDevP, VarP
+-- ArithmeticOps: Add, Subtract, Divide, Multiply, Remainder
+-- FilterOps: NotEqual, GreaterThanOrEqual, Like, LessThanOrEqual, Equal, GreaterThan
+-- GroupBy: NoGroupByProjectColumns
+EXPLAIN SELECT   cint,
+         cstring1,
+         cboolean2,
+         ctimestamp2,
+         cdouble,
+         cfloat,
+         cbigint,
+         csmallint,
+         cboolean1,
+         (cint + csmallint),
+         (cbigint - ctinyint),
+         (-(cbigint)),
+         (-(cfloat)),
+         ((cbigint - ctinyint) + cbigint),
+         (cdouble / cdouble),
+         (-(cdouble)),
+         ((cint + csmallint) * (-(cbigint))),
+         ((-(cdouble)) + cbigint),
+         (-1.389 / ctinyint),
+         (cbigint % cdouble),
+         (-(csmallint)),
+         (csmallint + (cint + csmallint))
+FROM     alltypesorc
+WHERE    (((csmallint > -26.28)
+           AND (cstring2 LIKE 'ss'))
+          OR ((cdouble <= cbigint)
+              AND ((cstring1 >= 'ss')
+                   AND (cint != cdouble)))
+          OR (ctinyint = -89010)
+             OR ((cbigint <= cfloat)
+                 AND (-26.28 <= csmallint)))
+ORDER BY cboolean1, cstring1, ctimestamp2, cfloat, cbigint, cstring1, cdouble, cint, csmallint, cdouble;
+SELECT   cint,
+         cstring1,
+         cboolean2,
+         ctimestamp2,
+         cdouble,
+         cfloat,
+         cbigint,
+         csmallint,
+         cboolean1,
+         (cint + csmallint),
+         (cbigint - ctinyint),
+         (-(cbigint)),
+         (-(cfloat)),
+         ((cbigint - ctinyint) + cbigint),
+         (cdouble / cdouble),
+         (-(cdouble)),
+         ((cint + csmallint) * (-(cbigint))),
+         ((-(cdouble)) + cbigint),
+         (-1.389 / ctinyint),
+         (cbigint % cdouble),
+         (-(csmallint)),
+         (csmallint + (cint + csmallint))
+FROM     alltypesorc
+WHERE    (((csmallint > -26.28)
+           AND (cstring2 LIKE 'ss'))
+          OR ((cdouble <= cbigint)
+              AND ((cstring1 >= 'ss')
+                   AND (cint != cdouble)))
+          OR (ctinyint = -89010)
+             OR ((cbigint <= cfloat)
+                 AND (-26.28 <= csmallint)))
+ORDER BY cboolean1, cstring1, ctimestamp2, cfloat, cbigint, cstring1, cdouble, cint, csmallint, cdouble;
+
+-- TargetTypeClasses: Long, String, Double, Timestamp
+-- Functions: Avg, Min, StDevP, Sum, Var
+-- ArithmeticOps: Divide, Subtract, Multiply, Remainder
+-- FilterOps: GreaterThan, LessThan, LessThanOrEqual, GreaterThanOrEqual, Like
+-- GroupBy: NoGroupByProjectColumns
+EXPLAIN SELECT   ctimestamp1,
+         cstring2,
+         cdouble,
+         cfloat,
+         cbigint,
+         csmallint,
+         (cbigint / 3569),
+         (-257 - csmallint),
+         (-6432 * cfloat),
+         (-(cdouble)),
+         (cdouble * 10.175),
+         ((-6432 * cfloat) / cfloat),
+         (-(cfloat)),
+         (cint % csmallint),
+         (-(cdouble)),
+         (cdouble * (-(cdouble)))
+FROM     alltypesorc
+WHERE    (((-1.389 >= cint)
+           AND ((csmallint < ctinyint)
+                AND (-6432 > csmallint)))
+          OR ((cdouble >= cfloat)
+              AND (cstring2 <= 'a'))
+             OR ((cstring1 LIKE 'ss%')
+                 AND (10.175 > cbigint)))
+ORDER BY csmallint, cstring2, cdouble;
+SELECT   ctimestamp1,
+         cstring2,
+         cdouble,
+         cfloat,
+         cbigint,
+         csmallint,
+         (cbigint / 3569),
+         (-257 - csmallint),
+         (-6432 * cfloat),
+         (-(cdouble)),
+         (cdouble * 10.175),
+         ((-6432 * cfloat) / cfloat),
+         (-(cfloat)),
+         (cint % csmallint),
+         (-(cdouble)),
+         (cdouble * (-(cdouble)))
+FROM     alltypesorc
+WHERE    (((-1.389 >= cint)
+           AND ((csmallint < ctinyint)
+                AND (-6432 > csmallint)))
+          OR ((cdouble >= cfloat)
+              AND (cstring2 <= 'a'))
+             OR ((cstring1 LIKE 'ss%')
+                 AND (10.175 > cbigint)))
+ORDER BY csmallint, cstring2, cdouble;
+
+-- TargetTypeClasses: Double, String, Long
+-- Functions: StDev, Sum, VarP, Count
+-- ArithmeticOps: Remainder, Divide, Subtract
+-- FilterOps: GreaterThanOrEqual, Equal, LessThanOrEqual
+-- GroupBy: GroupBy
+EXPLAIN SELECT   csmallint,
+         (csmallint % -75),
+         STDDEV_SAMP(csmallint),
+         (-1.389 / csmallint),
+         SUM(cbigint),
+         ((csmallint % -75) / SUM(cbigint)),
+         (-((csmallint % -75))),
+         VAR_POP(ctinyint),
+         (-((-((csmallint % -75))))),
+         COUNT(*),
+         (COUNT(*) - -89010)
+FROM     alltypesorc
+WHERE    (((csmallint >= -257))
+          AND ((-6432 = csmallint)
+               OR ((cint >= cdouble)
+                   AND (ctinyint <= cint))))
+GROUP BY csmallint
+ORDER BY csmallint;
+SELECT   csmallint,
+         (csmallint % -75),
+         STDDEV_SAMP(csmallint),
+         (-1.389 / csmallint),
+         SUM(cbigint),
+         ((csmallint % -75) / SUM(cbigint)),
+         (-((csmallint % -75))),
+         VAR_POP(ctinyint),
+         (-((-((csmallint % -75))))),
+         COUNT(*),
+         (COUNT(*) - -89010)
+FROM     alltypesorc
+WHERE    (((csmallint >= -257))
+          AND ((-6432 = csmallint)
+               OR ((cint >= cdouble)
+                   AND (ctinyint <= cint))))
+GROUP BY csmallint
+ORDER BY csmallint;
+
+-- TargetTypeClasses: Long, Double, Timestamp
+-- Functions: Var, Count, Sum, VarP, StDevP
+-- ArithmeticOps: Multiply, Add, Subtract, Remainder
+-- FilterOps: GreaterThan, LessThan, Equal, LessThanOrEqual, GreaterThanOrEqual
+-- GroupBy: GroupBy
+EXPLAIN SELECT   cdouble,
+         VAR_SAMP(cdouble),
+         (2563.58 * VAR_SAMP(cdouble)),
+         (-(VAR_SAMP(cdouble))),
+         COUNT(cfloat),
+         ((2563.58 * VAR_SAMP(cdouble)) + -5638.15),
+         ((-(VAR_SAMP(cdouble))) * ((2563.58 * VAR_SAMP(cdouble)) + -5638.15)),
+         SUM(cfloat),
+         VAR_POP(cdouble),
+         (cdouble - (-(VAR_SAMP(cdouble)))),
+         STDDEV_POP(cdouble),
+         (cdouble + VAR_SAMP(cdouble)),
+         (cdouble * 762),
+         SUM(cdouble),
+         (-863.257 % (cdouble * 762)),
+         SUM(cdouble)
+FROM     alltypesorc
+WHERE    (((cdouble > 2563.58))
+          AND (((cbigint >= cint)
+                AND ((csmallint < cint)
+                     AND (cfloat < -5638.15)))
+               OR (2563.58 = ctinyint)
+                  OR ((cdouble <= cbigint)
+                      AND (-5638.15 > cbigint))))
+GROUP BY cdouble
+ORDER BY cdouble;
+SELECT   cdouble,
+         VAR_SAMP(cdouble),
+         (2563.58 * VAR_SAMP(cdouble)),
+         (-(VAR_SAMP(cdouble))),
+         COUNT(cfloat),
+         ((2563.58 * VAR_SAMP(cdouble)) + -5638.15),
+         ((-(VAR_SAMP(cdouble))) * ((2563.58 * VAR_SAMP(cdouble)) + -5638.15)),
+         SUM(cfloat),
+         VAR_POP(cdouble),
+         (cdouble - (-(VAR_SAMP(cdouble)))),
+         STDDEV_POP(cdouble),
+         (cdouble + VAR_SAMP(cdouble)),
+         (cdouble * 762),
+         SUM(cdouble),
+         (-863.257 % (cdouble * 762)),
+         SUM(cdouble)
+FROM     alltypesorc
+WHERE    (((cdouble > 2563.58))
+          AND (((cbigint >= cint)
+                AND ((csmallint < cint)
+                     AND (cfloat < -5638.15)))
+               OR (2563.58 = ctinyint)
+                  OR ((cdouble <= cbigint)
+                      AND (-5638.15 > cbigint))))
+GROUP BY cdouble
+ORDER BY cdouble;
+
+-- TargetTypeClasses: Bool, Timestamp, String, Double, Long
+-- Functions: StDevP, Avg, Count, Min, Var, VarP, Sum
+-- ArithmeticOps: Multiply, Subtract, Add, Divide, Remainder
+-- FilterOps: NotEqual, LessThan, Like, Equal, RLike
+-- GroupBy: GroupBy
+EXPLAIN SELECT   ctimestamp1,
+         cstring1,
+         STDDEV_POP(cint),
+         (STDDEV_POP(cint) * 10.175),
+         (-(STDDEV_POP(cint))),
+         AVG(csmallint),
+         (-(STDDEV_POP(cint))),
+         (-26.28 - STDDEV_POP(cint)),
+         COUNT(*),
+         (-(COUNT(*))),
+         ((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))),
+         MIN(ctinyint),
+         (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*)))),
+         (-((STDDEV_POP(cint) * 10.175))),
+         VAR_SAMP(csmallint),
+         (VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))),
+         (-((-(STDDEV_POP(cint))))),
+         ((-(COUNT(*))) / STDDEV_POP(cint)),
+         VAR_POP(cfloat),
+         (10.175 / AVG(csmallint)),
+         AVG(cint),
+         VAR_SAMP(cfloat),
+         ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) - (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))),
+         (-((-((STDDEV_POP(cint) * 10.175))))),
+         AVG(cfloat),
+         (((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) - (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) * 10.175),
+         (10.175 % (10.175 / AVG(csmallint))),
+         (-(MIN(ctinyint))),
+         MIN(cdouble),
+         VAR_POP(csmallint),
+         (-(((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))))),
+         ((-(STDDEV_POP(cint))) % AVG(cfloat)),
+         (-26.28 / (-(MIN(ctinyint)))),
+         STDDEV_POP(ctinyint),
+         SUM(cint),
+         ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) / VAR_POP(cfloat)),
+         (-((-(COUNT(*))))),
+         COUNT(*),
+         ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) % -26.28)
+FROM     alltypesorc
+WHERE    (((ctimestamp1 != 0))
+          AND ((((-257 != ctinyint)
+                 AND (cboolean2 IS NOT NULL))
+                AND ((cstring1 RLIKE '.*ss')
+                     AND (-10669 < ctimestamp1)))
+               OR (ctimestamp2 = -10669)
+               OR ((ctimestamp1 < 0)
+                   AND (cstring2 LIKE '%b%'))
+                  OR (cdouble = cint)
+                     OR ((cboolean1 IS NULL)
+                         AND (cfloat < cint))))
+GROUP BY ctimestamp1, cstring1;
+SELECT   ctimestamp1,
+         cstring1,
+         STDDEV_POP(cint),
+         (STDDEV_POP(cint) * 10.175),
+         (-(STDDEV_POP(cint))),
+         AVG(csmallint),
+         (-(STDDEV_POP(cint))),
+         (-26.28 - STDDEV_POP(cint)),
+         COUNT(*),
+         (-(COUNT(*))),
+         ((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))),
+         MIN(ctinyint),
+         (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*)))),
+         (-((STDDEV_POP(cint) * 10.175))),
+         VAR_SAMP(csmallint),
+         (VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))),
+         (-((-(STDDEV_POP(cint))))),
+         ((-(COUNT(*))) / STDDEV_POP(cint)),
+         VAR_POP(cfloat),
+         (10.175 / AVG(csmallint)),
+         AVG(cint),
+         VAR_SAMP(cfloat),
+         ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) - (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))),
+         (-((-((STDDEV_POP(cint) * 10.175))))),
+         AVG(cfloat),
+         (((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) - (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) * 10.175),
+         (10.175 % (10.175 / AVG(csmallint))),
+         (-(MIN(ctinyint))),
+         MIN(cdouble),
+         VAR_POP(csmallint),
+         (-(((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))))),
+         ((-(STDDEV_POP(cint))) % AVG(cfloat)),
+         (-26.28 / (-(MIN(ctinyint)))),
+         STDDEV_POP(ctinyint),
+         SUM(cint),
+         ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) / VAR_POP(cfloat)),
+         (-((-(COUNT(*))))),
+         COUNT(*),
+         ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) % -26.28)
+FROM     alltypesorc
+WHERE    (((ctimestamp1 != 0))
+          AND ((((-257 != ctinyint)
+                 AND (cboolean2 IS NOT NULL))
+                AND ((cstring1 RLIKE '.*ss')
+                     AND (-10669 < ctimestamp1)))
+               OR (ctimestamp2 = -10669)
+               OR ((ctimestamp1 < 0)
+                   AND (cstring2 LIKE '%b%'))
+                  OR (cdouble = cint)
+                     OR ((cboolean1 IS NULL)
+                         AND (cfloat < cint))))
+GROUP BY ctimestamp1, cstring1;
+
+-- TargetTypeClasses: Double, Long, String, Timestamp, Bool
+-- Functions: Max, Sum, Var, Avg, Min, VarP, StDev, StDevP
+-- ArithmeticOps: Divide, Subtract, Remainder, Add, Multiply
+-- FilterOps: GreaterThan, LessThanOrEqual, Equal, LessThan, GreaterThanOrEqual, NotEqual, Like, RLike
+-- GroupBy: GroupBy
+EXPLAIN SELECT   cboolean1,
+         MAX(cfloat),
+         (-(MAX(cfloat))),
+         (-26.28 / MAX(cfloat)),
+         SUM(cbigint),
+         (SUM(cbigint) - 10.175),
+         VAR_SAMP(cint),
+         (VAR_SAMP(cint) % MAX(cfloat)),
+         (10.175 + (-(MAX(cfloat)))),
+         AVG(cdouble),
+         ((SUM(cbigint) - 10.175) + VAR_SAMP(cint)),
+         MIN(cbigint),
+         VAR_POP(cbigint),
+         (-((10.175 + (-(MAX(cfloat)))))),
+         (79.553 / VAR_POP(cbigint)),
+         (VAR_SAMP(cint) % (79.553 / VAR_POP(cbigint))),
+         (-((10.175 + (-(MAX(cfloat)))))),
+         SUM(cint),
+         STDDEV_SAMP(ctinyint),
+         (-1.389 * MIN(cbigint)),
+         (SUM(cint) - (-1.389 * MIN(cbigint))),
+         STDDEV_POP(csmallint),
+         (-((SUM(cint) - (-1.389 * MIN(cbigint))))),
+         AVG(cint),
+         (-(AVG(cint))),
+         (AVG(cint) * SUM(cint))
+FROM     alltypesorc
+WHERE    (((cboolean1 IS NOT NULL))
+          AND (((cdouble < csmallint)
+                AND ((cboolean2 = cboolean1)
+                     AND (cbigint <= -863.257)))
+               OR ((cint >= -257)
+                   AND ((cstring1 IS NOT NULL)
+                        AND (cboolean1 >= 1)))
+               OR (cstring2 RLIKE 'b')
+                  OR ((csmallint >= ctinyint)
+                      AND (ctimestamp2 IS NULL))))
+GROUP BY cboolean1
+ORDER BY cboolean1;
+SELECT   cboolean1,
+         MAX(cfloat),
+         (-(MAX(cfloat))),
+         (-26.28 / MAX(cfloat)),
+         SUM(cbigint),
+         (SUM(cbigint) - 10.175),
+         VAR_SAMP(cint),
+         (VAR_SAMP(cint) % MAX(cfloat)),
+         (10.175 + (-(MAX(cfloat)))),
+         AVG(cdouble),
+         ((SUM(cbigint) - 10.175) + VAR_SAMP(cint)),
+         MIN(cbigint),
+         VAR_POP(cbigint),
+         (-((10.175 + (-(MAX(cfloat)))))),
+         (79.553 / VAR_POP(cbigint)),
+         (VAR_SAMP(cint) % (79.553 / VAR_POP(cbigint))),
+         (-((10.175 + (-(MAX(cfloat)))))),
+         SUM(cint),
+         STDDEV_SAMP(ctinyint),
+         (-1.389 * MIN(cbigint)),
+         (SUM(cint) - (-1.389 * MIN(cbigint))),
+         STDDEV_POP(csmallint),
+         (-((SUM(cint) - (-1.389 * MIN(cbigint))))),
+         AVG(cint),
+         (-(AVG(cint))),
+         (AVG(cint) * SUM(cint))
+FROM     alltypesorc
+WHERE    (((cboolean1 IS NOT NULL))
+          AND (((cdouble < csmallint)
+                AND ((cboolean2 = cboolean1)
+                     AND (cbigint <= -863.257)))
+               OR ((cint >= -257)
+                   AND ((cstring1 IS NOT NULL)
+                        AND (cboolean1 >= 1)))
+               OR (cstring2 RLIKE 'b')
+                  OR ((csmallint >= ctinyint)
+                      AND (ctimestamp2 IS NULL))))
+GROUP BY cboolean1
+ORDER BY cboolean1;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_bucketmapjoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_bucketmapjoin1.q
new file mode 100644
index 0000000000000000000000000000000000000000..e309713795af8970f8f480333ddb3c54d7c86b5e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_bucketmapjoin1.q
@@ -0,0 +1,46 @@
+create table vsmb_bucket_1(key int, value string) 
+  CLUSTERED BY (key) 
+  SORTED BY (key) INTO 1 BUCKETS 
+  STORED AS ORC;
+create table vsmb_bucket_2(key int, value string) 
+  CLUSTERED BY (key) 
+  SORTED BY (key) INTO 1 BUCKETS 
+  STORED AS ORC;
+
+create table vsmb_bucket_RC(key int, value string) 
+  CLUSTERED BY (key) 
+  SORTED BY (key) INTO 1 BUCKETS 
+  STORED AS RCFILE;
+
+create table vsmb_bucket_TXT(key int, value string) 
+  CLUSTERED BY (key) 
+  SORTED BY (key) INTO 1 BUCKETS 
+  STORED AS TEXTFILE;
+  
+insert into table vsmb_bucket_1 select cint, cstring1 from alltypesorc limit 2;
+insert into table vsmb_bucket_2 select cint, cstring1 from alltypesorc limit 2;
+insert into table vsmb_bucket_RC select cint, cstring1 from alltypesorc limit 2;
+insert into table vsmb_bucket_TXT select cint, cstring1 from alltypesorc limit 2;  
+
+set hive.vectorized.execution.enabled=true;
+set hive.optimize.bucketmapjoin = true;
+set hive.optimize.bucketmapjoin.sortedmerge = true;
+set hive.auto.convert.sortmerge.join.noconditionaltask = true;
+set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat;
+
+explain
+select /*+MAPJOIN(a)*/ * from vsmb_bucket_1 a join vsmb_bucket_2 b on a.key = b.key;
+select /*+MAPJOIN(a)*/ * from vsmb_bucket_1 a join vsmb_bucket_2 b on a.key = b.key;
+
+explain
+select /*+MAPJOIN(b)*/ * from vsmb_bucket_1 a join vsmb_bucket_RC b on a.key = b.key;
+select /*+MAPJOIN(b)*/ * from vsmb_bucket_1 a join vsmb_bucket_RC b on a.key = b.key;
+
+-- RC file does not yet provide the vectorized CommonRCFileformat out-of-the-box
+-- explain
+-- select /*+MAPJOIN(b)*/ * from vsmb_bucket_RC a join vsmb_bucket_2 b on a.key = b.key;
+-- select /*+MAPJOIN(b)*/ * from vsmb_bucket_RC a join vsmb_bucket_2 b on a.key = b.key;
+
+explain
+select /*+MAPJOIN(b)*/ * from vsmb_bucket_1 a join vsmb_bucket_TXT b on a.key = b.key;
+select /*+MAPJOIN(b)*/ * from vsmb_bucket_1 a join vsmb_bucket_TXT b on a.key = b.key;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_case.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_case.q
new file mode 100644
index 0000000000000000000000000000000000000000..e448d51f6bc305cdd024e7248ec7977065dae71e
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_case.q
@@ -0,0 +1,37 @@
+set hive.vectorized.execution.enabled = true
+;
+explain
+select 
+  csmallint,
+  case 
+    when csmallint = 418 then "a"
+    when csmallint = 12205 then "b"
+    else "c"
+  end,
+  case csmallint
+    when 418 then "a"
+    when 12205 then "b"
+    else "c"
+  end
+from alltypesorc
+where csmallint = 418
+or csmallint = 12205
+or csmallint = 10583
+;
+select 
+  csmallint,
+  case 
+    when csmallint = 418 then "a"
+    when csmallint = 12205 then "b"
+    else "c"
+  end,
+  case csmallint
+    when 418 then "a"
+    when 12205 then "b"
+    else "c"
+  end
+from alltypesorc
+where csmallint = 418
+or csmallint = 12205
+or csmallint = 10583
+;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q
new file mode 100644
index 0000000000000000000000000000000000000000..3f818b18534af936cccb78d247170810a50e433f
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q
@@ -0,0 +1,149 @@
+SET hive.vectorized.execution.enabled = true;
+
+-- Test type casting in vectorized mode to verify end-to-end functionality.
+
+explain 
+select 
+-- to boolean
+   cast (ctinyint as boolean)
+  ,cast (csmallint as boolean)
+  ,cast (cint as boolean)
+  ,cast (cbigint as boolean)
+  ,cast (cfloat as boolean)
+  ,cast (cdouble as boolean)
+  ,cast (cboolean1 as boolean)
+  ,cast (cbigint * 0 as boolean)
+  ,cast (ctimestamp1 as boolean)
+  ,cast (cstring1 as boolean)
+-- to int family
+  ,cast (ctinyint as int)
+  ,cast (csmallint as int)
+  ,cast (cint as int)
+  ,cast (cbigint as int)
+  ,cast (cfloat as int)
+  ,cast (cdouble as int)
+  ,cast (cboolean1 as int)
+  ,cast (ctimestamp1 as int)
+  ,cast (cstring1 as int)
+  ,cast (substr(cstring1, 1, 1) as int)
+  ,cast (cfloat as tinyint)
+  ,cast (cfloat as smallint)
+  ,cast (cfloat as bigint)
+-- to float family
+  ,cast (ctinyint as double)
+  ,cast (csmallint as double)
+  ,cast (cint as double)
+  ,cast (cbigint as double)
+  ,cast (cfloat as double)
+  ,cast (cdouble as double)
+  ,cast (cboolean1 as double)
+  ,cast (ctimestamp1 as double)
+  ,cast (cstring1 as double)
+  ,cast (substr(cstring1, 1, 1) as double)
+  ,cast (cint as float)
+  ,cast (cdouble as float)
+-- to timestamp
+  ,cast (ctinyint as timestamp)
+  ,cast (csmallint as timestamp)
+  ,cast (cint as timestamp)
+  ,cast (cbigint as timestamp)
+  ,cast (cfloat as timestamp)
+  ,cast (cdouble as timestamp)
+  ,cast (cboolean1 as timestamp)
+  ,cast (cbigint * 0 as timestamp)
+  ,cast (ctimestamp1 as timestamp)
+  ,cast (cstring1 as timestamp)
+  ,cast (substr(cstring1, 1, 1) as timestamp)
+-- to string
+  ,cast (ctinyint as string)
+  ,cast (csmallint as string)
+  ,cast (cint as string)
+  ,cast (cbigint as string)
+  ,cast (cfloat as string)
+  ,cast (cdouble as string)
+  ,cast (cboolean1 as string)
+  ,cast (cbigint * 0 as string)
+  ,cast (ctimestamp1 as string)
+  ,cast (cstring1 as string)
+-- nested and expression arguments
+  ,cast (cast (cfloat as int) as float)
+  ,cast (cint * 2 as double)
+  ,cast (sin(cfloat) as string)
+  ,cast (cint as float) + cast(cboolean1 as double)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 250 = 0;
+
+
+select 
+-- to boolean
+   cast (ctinyint as boolean)
+  ,cast (csmallint as boolean)
+  ,cast (cint as boolean)
+  ,cast (cbigint as boolean)
+  ,cast (cfloat as boolean)
+  ,cast (cdouble as boolean)
+  ,cast (cboolean1 as boolean)
+  ,cast (cbigint * 0 as boolean)
+  ,cast (ctimestamp1 as boolean)
+  ,cast (cstring1 as boolean)
+-- to int family
+  ,cast (ctinyint as int)
+  ,cast (csmallint as int)
+  ,cast (cint as int)
+  ,cast (cbigint as int)
+  ,cast (cfloat as int)
+  ,cast (cdouble as int)
+  ,cast (cboolean1 as int)
+  ,cast (ctimestamp1 as int)
+  ,cast (cstring1 as int)
+  ,cast (substr(cstring1, 1, 1) as int)
+  ,cast (cfloat as tinyint)
+  ,cast (cfloat as smallint)
+  ,cast (cfloat as bigint)
+-- to float family
+  ,cast (ctinyint as double)
+  ,cast (csmallint as double)
+  ,cast (cint as double)
+  ,cast (cbigint as double)
+  ,cast (cfloat as double)
+  ,cast (cdouble as double)
+  ,cast (cboolean1 as double)
+  ,cast (ctimestamp1 as double)
+  ,cast (cstring1 as double)
+  ,cast (substr(cstring1, 1, 1) as double)
+  ,cast (cint as float)
+  ,cast (cdouble as float)
+-- to timestamp
+  ,cast (ctinyint as timestamp)
+  ,cast (csmallint as timestamp)
+  ,cast (cint as timestamp)
+  ,cast (cbigint as timestamp)
+  ,cast (cfloat as timestamp)
+  ,cast (cdouble as timestamp)
+  ,cast (cboolean1 as timestamp)
+  ,cast (cbigint * 0 as timestamp)
+  ,cast (ctimestamp1 as timestamp)
+  ,cast (cstring1 as timestamp)
+  ,cast (substr(cstring1, 1, 1) as timestamp)
+-- to string
+  ,cast (ctinyint as string)
+  ,cast (csmallint as string)
+  ,cast (cint as string)
+  ,cast (cbigint as string)
+  ,cast (cfloat as string)
+  ,cast (cdouble as string)
+  ,cast (cboolean1 as string)
+  ,cast (cbigint * 0 as string)
+  ,cast (ctimestamp1 as string)
+  ,cast (cstring1 as string)
+-- nested and expression arguments
+  ,cast (cast (cfloat as int) as float)
+  ,cast (cint * 2 as double)
+  ,cast (sin(cfloat) as string)
+  ,cast (cint as float) + cast(cboolean1 as double)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 250 = 0;
+
+ 
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_context.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_context.q
new file mode 100644
index 0000000000000000000000000000000000000000..381e4255ca51cc34dea56bf0ada574ad298b2575
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_context.q
@@ -0,0 +1,47 @@
+create table store(s_store_sk int, s_city string)
+stored as orc;
+insert overwrite table store
+select cint, cstring1
+from alltypesorc
+where cint not in (
+-3728, -563, 762, 6981, 253665376, 528534767, 626923679);
+create table store_sales(ss_store_sk int, ss_hdemo_sk int, ss_net_profit double)
+stored as orc;
+insert overwrite table store_sales
+select cint, cint, cdouble
+from alltypesorc
+where cint not in (
+-3728, -563, 762, 6981, 253665376, 528534767, 626923679);
+create table household_demographics(hd_demo_sk int)
+stored as orc;
+insert overwrite table household_demographics
+select cint
+from alltypesorc
+where cint not in (
+-3728, -563, 762, 6981, 253665376, 528534767, 626923679);
+set hive.auto.convert.join=true;
+set hive.vectorized.execution.enabled=true;
+
+
+explain 
+select store.s_city, ss_net_profit
+from store_sales
+JOIN store ON store_sales.ss_store_sk = store.s_store_sk
+JOIN household_demographics ON store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+limit 100
+;
+
+select store.s_city, ss_net_profit
+from store_sales
+JOIN store ON store_sales.ss_store_sk = store.s_store_sk
+JOIN household_demographics ON store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk
+limit 100
+;
+
+set hive.auto.convert.join=false;
+set hive.vectorized.execution.enabled=false;
+
+drop table store;
+drop table store_sales;
+drop table household_demographics;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_date_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_date_funcs.q
new file mode 100644
index 0000000000000000000000000000000000000000..b7aa3c28ac059722eb094c0bb3411bdea0857102
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_date_funcs.q
@@ -0,0 +1,122 @@
+SET hive.vectorized.execution.enabled = true;
+
+-- Test timestamp functions in vectorized mode to verify they run correctly end-to-end.
+
+CREATE TABLE date_udf_flight (
+  origin_city_name STRING,
+  dest_city_name STRING,
+  fl_date DATE,
+  arr_delay FLOAT,
+  fl_num INT
+);
+LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_udf_flight;
+
+CREATE TABLE date_udf_flight_orc (
+  fl_date DATE,
+  fl_time TIMESTAMP
+) STORED AS ORC;
+
+INSERT INTO TABLE date_udf_flight_orc SELECT fl_date, to_utc_timestamp(fl_date, 'America/Los_Angeles') FROM date_udf_flight;
+
+SELECT * FROM date_udf_flight_orc;
+
+EXPLAIN SELECT
+  to_unix_timestamp(fl_time),
+  year(fl_time),
+  month(fl_time),
+  day(fl_time),
+  dayofmonth(fl_time),
+  weekofyear(fl_time),
+  date(fl_time),
+  to_date(fl_time),
+  date_add(fl_time, 2),
+  date_sub(fl_time, 2),
+  datediff(fl_time, "2000-01-01")
+FROM date_udf_flight_orc;
+
+SELECT
+  to_unix_timestamp(fl_time),
+  year(fl_time),
+  month(fl_time),
+  day(fl_time),
+  dayofmonth(fl_time),
+  weekofyear(fl_time),
+  date(fl_time),
+  to_date(fl_time),
+  date_add(fl_time, 2),
+  date_sub(fl_time, 2),
+  datediff(fl_time, "2000-01-01")
+FROM date_udf_flight_orc;
+
+EXPLAIN SELECT
+  to_unix_timestamp(fl_date),
+  year(fl_date),
+  month(fl_date),
+  day(fl_date),
+  dayofmonth(fl_date),
+  weekofyear(fl_date),
+  date(fl_date),
+  to_date(fl_date),
+  date_add(fl_date, 2),
+  date_sub(fl_date, 2),
+  datediff(fl_date, "2000-01-01")
+FROM date_udf_flight_orc;
+
+SELECT
+  to_unix_timestamp(fl_date),
+  year(fl_date),
+  month(fl_date),
+  day(fl_date),
+  dayofmonth(fl_date),
+  weekofyear(fl_date),
+  date(fl_date),
+  to_date(fl_date),
+  date_add(fl_date, 2),
+  date_sub(fl_date, 2),
+  datediff(fl_date, "2000-01-01")
+FROM date_udf_flight_orc;
+
+EXPLAIN SELECT
+  year(fl_time) = year(fl_date),
+  month(fl_time) = month(fl_date),
+  day(fl_time) = day(fl_date),
+  dayofmonth(fl_time) = dayofmonth(fl_date),
+  weekofyear(fl_time) = weekofyear(fl_date),
+  date(fl_time) = date(fl_date),
+  to_date(fl_time) = to_date(fl_date),
+  date_add(fl_time, 2) = date_add(fl_date, 2),
+  date_sub(fl_time, 2) = date_sub(fl_date, 2),
+  datediff(fl_time, "2000-01-01") = datediff(fl_date, "2000-01-01")
+FROM date_udf_flight_orc;
+
+-- Should all be true or NULL
+SELECT
+  year(fl_time) = year(fl_date),
+  month(fl_time) = month(fl_date),
+  day(fl_time) = day(fl_date),
+  dayofmonth(fl_time) = dayofmonth(fl_date),
+  weekofyear(fl_time) = weekofyear(fl_date),
+  date(fl_time) = date(fl_date),
+  to_date(fl_time) = to_date(fl_date),
+  date_add(fl_time, 2) = date_add(fl_date, 2),
+  date_sub(fl_time, 2) = date_sub(fl_date, 2),
+  datediff(fl_time, "2000-01-01") = datediff(fl_date, "2000-01-01")
+FROM date_udf_flight_orc;
+
+EXPLAIN SELECT 
+  fl_date, 
+  to_date(date_add(fl_date, 2)), 
+  to_date(date_sub(fl_date, 2)),
+  datediff(fl_date, date_add(fl_date, 2)), 
+  datediff(fl_date, date_sub(fl_date, 2)),
+  datediff(date_add(fl_date, 2), date_sub(fl_date, 2)) 
+FROM date_udf_flight_orc LIMIT 10;
+
+SELECT 
+  fl_date, 
+  to_date(date_add(fl_date, 2)), 
+  to_date(date_sub(fl_date, 2)),
+  datediff(fl_date, date_add(fl_date, 2)), 
+  datediff(fl_date, date_sub(fl_date, 2)),
+  datediff(date_add(fl_date, 2), date_sub(fl_date, 2)) 
+FROM date_udf_flight_orc LIMIT 10;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_distinct_gby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_distinct_gby.q
new file mode 100644
index 0000000000000000000000000000000000000000..6e622007e5dcbda7510f7806de517f0b7965dce6
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_distinct_gby.q
@@ -0,0 +1,12 @@
+SET hive.vectorized.execution.enabled=true;
+
+SET hive.map.groupby.sorted=true;
+
+create table dtest(a int, b int) clustered by (a) sorted by (a) into 1 buckets stored as orc;
+insert into table dtest select c,b from (select array(300,300,300,300,300) as a, 1 as b from src limit 1) y lateral view  explode(a) t1 as c;
+
+explain select sum(distinct a), count(distinct a) from dtest;
+select sum(distinct a), count(distinct a) from dtest;
+
+explain select sum(distinct cint), count(distinct cint), avg(distinct cint), std(distinct cint) from alltypesorc;
+select sum(distinct cint), count(distinct cint), avg(distinct cint), std(distinct cint) from alltypesorc;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_mapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_mapjoin.q
new file mode 100644
index 0000000000000000000000000000000000000000..f390c2caafeb2af4e993b132f7c4b1860aeaaed6
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_mapjoin.q
@@ -0,0 +1,12 @@
+SET hive.vectorized.execution.enabled=true;
+SET hive.auto.convert.join=true;
+SET hive.auto.convert.join.nonconditionaltask=true;
+SET hive.auto.convert.join.nonconditionaltask.size=1000000000;
+
+EXPLAIN SELECT COUNT(t1.cint), MAX(t2.cint), MIN(t1.cint), AVG(t1.cint+t2.cint)
+  FROM alltypesorc t1
+  JOIN alltypesorc t2 ON t1.cint = t2.cint;
+
+SELECT COUNT(t1.cint), MAX(t2.cint), MIN(t1.cint), AVG(t1.cint+t2.cint)
+  FROM alltypesorc t1
+  JOIN alltypesorc t2 ON t1.cint = t2.cint;  
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_math_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_math_funcs.q
new file mode 100644
index 0000000000000000000000000000000000000000..d6b082467938f973eb7cd54a0821dbf8be0dde67
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_math_funcs.q
@@ -0,0 +1,107 @@
+SET hive.vectorized.execution.enabled = true;
+
+-- Test math functions in vectorized mode to verify they run correctly end-to-end.
+
+explain 
+select
+   cdouble
+  ,Round(cdouble, 2)
+  ,Floor(cdouble)
+  ,Ceil(cdouble)
+  ,Rand()
+  ,Rand(98007)
+  ,Exp(ln(cdouble))
+  ,Ln(cdouble)  
+  ,Ln(cfloat)
+  ,Log10(cdouble)
+  -- Use log2 as a representative function to test all input types.
+  ,Log2(cdouble)
+  -- Use 15601.0 to test zero handling, as there are no zeroes in the table
+  ,Log2(cdouble - 15601.0)
+  ,Log2(cfloat)
+  ,Log2(cbigint)
+  ,Log2(cint)
+  ,Log2(csmallint)
+  ,Log2(ctinyint)
+  ,Log(2.0, cdouble)
+  ,Pow(log2(cdouble), 2.0)  
+  ,Power(log2(cdouble), 2.0)
+  ,Sqrt(cdouble)
+  ,Sqrt(cbigint)
+  ,Bin(cbigint)
+  ,Hex(cdouble)
+  ,Conv(cbigint, 10, 16)
+  ,Abs(cdouble)
+  ,Abs(ctinyint)
+  ,Pmod(cint, 3)
+  ,Sin(cdouble)
+  ,Asin(cdouble)
+  ,Cos(cdouble)
+  ,ACos(cdouble)
+  ,Atan(cdouble)
+  ,Degrees(cdouble)
+  ,Radians(cdouble)
+  ,Positive(cdouble)
+  ,Positive(cbigint)
+  ,Negative(cdouble)
+  ,Sign(cdouble)
+  ,Sign(cbigint)
+  -- Test nesting
+  ,cos(-sin(log(cdouble)) + 3.14159)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 500 = 0
+-- test use of a math function in the WHERE clause
+and sin(cfloat) >= -1.0;
+
+select
+   cdouble
+  ,Round(cdouble, 2)
+  ,Floor(cdouble)
+  ,Ceil(cdouble)
+  -- Omit rand() from runtime test because it's nondeterministic.
+  -- ,Rand()
+  ,Rand(98007)
+  ,Exp(ln(cdouble))
+  ,Ln(cdouble)  
+  ,Ln(cfloat)
+  ,Log10(cdouble)
+  -- Use log2 as a representative function to test all input types.
+  ,Log2(cdouble)
+  -- Use 15601.0 to test zero handling, as there are no zeroes in the table
+  ,Log2(cdouble - 15601.0)
+  ,Log2(cfloat)
+  ,Log2(cbigint)
+  ,Log2(cint)
+  ,Log2(csmallint)
+  ,Log2(ctinyint)
+  ,Log(2.0, cdouble)
+  ,Pow(log2(cdouble), 2.0)  
+  ,Power(log2(cdouble), 2.0)
+  ,Sqrt(cdouble)
+  ,Sqrt(cbigint)
+  ,Bin(cbigint)
+  ,Hex(cdouble)
+  ,Conv(cbigint, 10, 16)
+  ,Abs(cdouble)
+  ,Abs(ctinyint)
+  ,Pmod(cint, 3)
+  ,Sin(cdouble)
+  ,Asin(cdouble)
+  ,Cos(cdouble)
+  ,ACos(cdouble)
+  ,Atan(cdouble)
+  ,Degrees(cdouble)
+  ,Radians(cdouble)
+  ,Positive(cdouble)
+  ,Positive(cbigint)
+  ,Negative(cdouble)
+  ,Sign(cdouble)
+  ,Sign(cbigint)
+  -- Test nesting
+  ,cos(-sin(log(cdouble)) + 3.14159)
+from alltypesorc
+-- limit output to a reasonably small number of rows
+where cbigint % 500 = 0
+-- test use of a math function in the WHERE clause
+and sin(cfloat) >= -1.0;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_nested_mapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_nested_mapjoin.q
new file mode 100644
index 0000000000000000000000000000000000000000..ce4227cf0a700f0ea50b30cebd5464e310b37ba6
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_nested_mapjoin.q
@@ -0,0 +1,8 @@
+SET hive.vectorized.execution.enabled=true;
+SET hive.auto.convert.join=true;
+SET hive.auto.convert.join.nonconditionaltask=true;
+SET hive.auto.convert.join.nonconditionaltask.size=1000000000;
+
+explain select sum(t1.td) from (select  v1.csmallint as tsi, v1.cdouble as td from alltypesorc v1, alltypesorc v2 where v1.ctinyint=v2.ctinyint) t1 join alltypesorc v3 on t1.tsi=v3.csmallint;
+
+select sum(t1.td) from (select  v1.csmallint as tsi, v1.cdouble as td from alltypesorc v1, alltypesorc v2 where v1.ctinyint=v2.ctinyint) t1 join alltypesorc v3 on t1.tsi=v3.csmallint;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_rcfile_columnar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_rcfile_columnar.q
new file mode 100644
index 0000000000000000000000000000000000000000..488d2f38859b3228ad9cdba274f39f7adf3c8845
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_rcfile_columnar.q
@@ -0,0 +1,18 @@
+--This query must pass even when vectorized reader is not available for
+--RC files. The query must fall back to the non-vector mode and run successfully.
+
+CREATE table columnTable (key STRING, value STRING)
+ROW FORMAT SERDE
+  'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'
+STORED AS
+  INPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileInputFormat'
+  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileOutputFormat';
+
+FROM src
+INSERT OVERWRITE TABLE columnTable SELECT src.key, src.value LIMIT 10;
+describe columnTable;
+
+SET hive.vectorized.execution.enabled=true;
+
+SELECT key, value FROM columnTable ORDER BY key;
+
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_shufflejoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_shufflejoin.q
new file mode 100644
index 0000000000000000000000000000000000000000..6b60aa08c581b2354bc73feb2ea31082f2349862
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_shufflejoin.q
@@ -0,0 +1,10 @@
+SET hive.vectorized.execution.enabled=true;
+SET hive.auto.convert.join=false;
+
+EXPLAIN SELECT COUNT(t1.cint), MAX(t2.cint), MIN(t1.cint), AVG(t1.cint+t2.cint)
+  FROM alltypesorc t1
+  JOIN alltypesorc t2 ON t1.cint = t2.cint;
+
+SELECT COUNT(t1.cint), MAX(t2.cint), MIN(t1.cint), AVG(t1.cint+t2.cint)
+  FROM alltypesorc t1
+  JOIN alltypesorc t2 ON t1.cint = t2.cint;  
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_string_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_string_funcs.q
new file mode 100644
index 0000000000000000000000000000000000000000..96fe53da1ea35b1f11815068e982031a89ac590d
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_string_funcs.q
@@ -0,0 +1,46 @@
+SET hive.vectorized.execution.enabled = true;
+
+-- Test string functions in vectorized mode to verify end-to-end functionality.
+
+explain 
+select 
+   substr(cstring1, 1, 2)
+  ,substr(cstring1, 2)
+  ,lower(cstring1)
+  ,upper(cstring1)
+  ,ucase(cstring1)
+  ,length(cstring1)
+  ,trim(cstring1)
+  ,ltrim(cstring1)
+  ,rtrim(cstring1)
+  ,concat(cstring1, cstring2)
+  ,concat('>', cstring1)
+  ,concat(cstring1, '<')
+  ,concat(substr(cstring1, 1, 2), substr(cstring2, 1, 2))
+from alltypesorc
+-- Limit the number of rows of output to a reasonable amount.
+where cbigint % 237 = 0
+-- Test function use in the WHERE clause.
+and length(substr(cstring1, 1, 2)) <= 2
+and cstring1 like '%';
+ 
+select 
+   substr(cstring1, 1, 2)
+  ,substr(cstring1, 2)
+  ,lower(cstring1)
+  ,upper(cstring1)
+  ,ucase(cstring1)
+  ,length(cstring1)
+  ,trim(cstring1)
+  ,ltrim(cstring1)
+  ,rtrim(cstring1)
+  ,concat(cstring1, cstring2)
+  ,concat('>', cstring1)
+  ,concat(cstring1, '<')
+  ,concat(substr(cstring1, 1, 2), substr(cstring2, 1, 2))
+from alltypesorc
+-- Limit the number of rows of output to a reasonable amount.
+where cbigint % 237 = 0
+-- Test function use in the WHERE clause.
+and length(substr(cstring1, 1, 2)) <= 2
+and cstring1 like '%';
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q
new file mode 100644
index 0000000000000000000000000000000000000000..95eedd3b581d69331836208cd7e4a79d816716cd
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q
@@ -0,0 +1,124 @@
+SET hive.vectorized.execution.enabled = true;
+
+-- Test timestamp functions in vectorized mode to verify they run correctly end-to-end.
+
+CREATE TABLE alltypesorc_string(ctimestamp1 timestamp, stimestamp1 string) STORED AS ORC;
+
+INSERT OVERWRITE TABLE alltypesorc_string
+SELECT
+  to_utc_timestamp(ctimestamp1, 'America/Los_Angeles'),
+  CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING)
+FROM alltypesorc
+LIMIT 40;
+
+CREATE TABLE alltypesorc_wrong(stimestamp1 string) STORED AS ORC;
+
+INSERT INTO TABLE alltypesorc_wrong SELECT 'abcd' FROM alltypesorc LIMIT 1;
+INSERT INTO TABLE alltypesorc_wrong SELECT '2000:01:01 00-00-00' FROM alltypesorc LIMIT 1;
+INSERT INTO TABLE alltypesorc_wrong SELECT '0000-00-00 99:99:99' FROM alltypesorc LIMIT 1;
+
+EXPLAIN SELECT
+  to_unix_timestamp(ctimestamp1) AS c1,
+  year(ctimestamp1),
+  month(ctimestamp1),
+  day(ctimestamp1),
+  dayofmonth(ctimestamp1),
+  weekofyear(ctimestamp1),
+  hour(ctimestamp1),
+  minute(ctimestamp1),
+  second(ctimestamp1)
+FROM alltypesorc_string
+ORDER BY c1;
+
+SELECT
+  to_unix_timestamp(ctimestamp1) AS c1,
+  year(ctimestamp1),
+  month(ctimestamp1),
+  day(ctimestamp1),
+  dayofmonth(ctimestamp1),
+  weekofyear(ctimestamp1),
+  hour(ctimestamp1),
+  minute(ctimestamp1),
+  second(ctimestamp1)
+FROM alltypesorc_string
+ORDER BY c1;
+
+EXPLAIN SELECT
+  to_unix_timestamp(stimestamp1) AS c1,
+  year(stimestamp1),
+  month(stimestamp1),
+  day(stimestamp1),
+  dayofmonth(stimestamp1),
+  weekofyear(stimestamp1),
+  hour(stimestamp1),
+  minute(stimestamp1),
+  second(stimestamp1)
+FROM alltypesorc_string
+ORDER BY c1;
+
+SELECT
+  to_unix_timestamp(stimestamp1) AS c1,
+  year(stimestamp1),
+  month(stimestamp1),
+  day(stimestamp1),
+  dayofmonth(stimestamp1),
+  weekofyear(stimestamp1),
+  hour(stimestamp1),
+  minute(stimestamp1),
+  second(stimestamp1)
+FROM alltypesorc_string
+ORDER BY c1;
+
+EXPLAIN SELECT
+  to_unix_timestamp(ctimestamp1) = to_unix_timestamp(stimestamp1) AS c1,
+  year(ctimestamp1) = year(stimestamp1),
+  month(ctimestamp1) = month(stimestamp1),
+  day(ctimestamp1) = day(stimestamp1),
+  dayofmonth(ctimestamp1) = dayofmonth(stimestamp1),
+  weekofyear(ctimestamp1) = weekofyear(stimestamp1),
+  hour(ctimestamp1) = hour(stimestamp1),
+  minute(ctimestamp1) = minute(stimestamp1),
+  second(ctimestamp1) = second(stimestamp1)
+FROM alltypesorc_string
+ORDER BY c1;
+
+-- Should all be true or NULL
+SELECT
+  to_unix_timestamp(ctimestamp1) = to_unix_timestamp(stimestamp1) AS c1,
+  year(ctimestamp1) = year(stimestamp1),
+  month(ctimestamp1) = month(stimestamp1),
+  day(ctimestamp1) = day(stimestamp1),
+  dayofmonth(ctimestamp1) = dayofmonth(stimestamp1),
+  weekofyear(ctimestamp1) = weekofyear(stimestamp1),
+  hour(ctimestamp1) = hour(stimestamp1),
+  minute(ctimestamp1) = minute(stimestamp1),
+  second(ctimestamp1) = second(stimestamp1)
+FROM alltypesorc_string
+ORDER BY c1;
+
+-- Wrong format. Should all be NULL.
+EXPLAIN SELECT
+  to_unix_timestamp(stimestamp1) AS c1,
+  year(stimestamp1),
+  month(stimestamp1),
+  day(stimestamp1),
+  dayofmonth(stimestamp1),
+  weekofyear(stimestamp1),
+  hour(stimestamp1),
+  minute(stimestamp1),
+  second(stimestamp1)
+FROM alltypesorc_wrong
+ORDER BY c1;
+
+SELECT
+  to_unix_timestamp(stimestamp1) AS c1,
+  year(stimestamp1),
+  month(stimestamp1),
+  day(stimestamp1),
+  dayofmonth(stimestamp1),
+  weekofyear(stimestamp1),
+  hour(stimestamp1),
+  minute(stimestamp1),
+  second(stimestamp1)
+FROM alltypesorc_wrong
+ORDER BY c1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q
index 4e3d0572921d61d0cb15c1a1bb6c7488a8871fef..bc193554f91822d8bd86bd0bce111188049e32e4 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q
@@ -4,13 +4,13 @@ USE db1;
 CREATE TABLE table1 (key STRING, value STRING)
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt'
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt'
 OVERWRITE INTO TABLE table1;
 
 CREATE TABLE table2 (key STRING, value STRING)
 STORED AS TEXTFILE;
 
-LOAD DATA LOCAL INPATH '../data/files/kv1.txt'
+LOAD DATA LOCAL INPATH '../../data/files/kv1.txt'
 OVERWRITE INTO TABLE table2;
 
 -- relative reference, no alias
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q
index b0b078ec628b3997ab7e77ba86082b02fe86ebaa..95517c3bcd367ac6abae2d3e014dcd0bb7f97fd8 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q
@@ -1,11 +1,11 @@
 DROP TABLE IF EXISTS atab; 
 CREATE TABLE IF NOT EXISTS atab (ks_uid BIGINT, sr_uid STRING, sr_id STRING, tstamp STRING, m_id STRING, act STRING, at_sr_uid STRING, tstamp_type STRING, original_m_id STRING, original_tstamp STRING, registered_flag TINYINT, at_ks_uid BIGINT) PARTITIONED BY (dt STRING,nt STRING); 
-LOAD DATA LOCAL INPATH '../data/files/v1.txt' INTO TABLE atab PARTITION (dt='20130312', nt='tw');
-LOAD DATA LOCAL INPATH '../data/files/v1.txt' INTO TABLE atab PARTITION (dt='20130311', nt='tw');
+LOAD DATA LOCAL INPATH '../../data/files/v1.txt' INTO TABLE atab PARTITION (dt='20130312', nt='tw');
+LOAD DATA LOCAL INPATH '../../data/files/v1.txt' INTO TABLE atab PARTITION (dt='20130311', nt='tw');
 
 DROP TABLE IF EXISTS  mstab;
 CREATE TABLE  mstab(ks_uid INT, csc INT) PARTITIONED BY (dt STRING);
-LOAD DATA LOCAL INPATH '../data/files/v2.txt' INTO TABLE mstab PARTITION (dt='20130311');
+LOAD DATA LOCAL INPATH '../../data/files/v2.txt' INTO TABLE mstab PARTITION (dt='20130311');
 
 DROP VIEW IF EXISTS aa_view_tw;
 CREATE VIEW aa_view_tw AS SELECT ks_uid, sr_id, act, at_ks_uid, at_sr_uid, from_unixtime(CAST(CAST( tstamp as BIGINT)/1000 AS BIGINT),'yyyyMMdd') AS act_date, from_unixtime(CAST(CAST( original_tstamp AS BIGINT)/1000 AS BIGINT),'yyyyMMdd') AS content_creation_date FROM atab WHERE dt='20130312' AND nt='tw' AND ks_uid != at_ks_uid;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q
index a7297db6104c07e4a1f988e2a6667fd83a19f289..2f22145518c5613c432d95d822a0e97dfe14c5dc 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q
@@ -13,7 +13,7 @@ CREATE TABLE part(
     p_comment STRING
 );
 
-LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part;
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
 
 -- 1. testWindowing
 select p_mfgr, p_name, p_size,
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q
index 9c7625dcd786a9d99de90a8da7e9c5703dfc0df0..67cab9f7b273a8471c52d9d39d747e931cbb1776 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q
@@ -13,7 +13,7 @@ CREATE TABLE part(
     p_comment STRING
 );
 
-LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part;
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
 
 set hive.join.cache.size=1;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q
index 7c4ab386145d58fa3d37169c4055820772af53d6..24f9ff73a30a163b7b5f2dac1676f27caaa55f3c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q
@@ -13,7 +13,7 @@ CREATE TABLE part(
     p_comment STRING
 );
 
-LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part;
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
 
 -- 1. testQueryLevelPartitionColsNotInSelect
 select p_size,
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q
index 2c3339095f22b837d04b415fa3114acb93458a73..7e27c6b1c098d48a30fbd2eb9086f9785c6b07fa 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q
@@ -13,7 +13,7 @@ CREATE TABLE part(
     p_comment STRING
 );
 
-LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part;
+LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part;
 
 drop table over10k;
 
@@ -27,12 +27,12 @@ create table over10k(
            bo boolean,
            s string,
 	   ts timestamp, 
-           dec decimal,  
+           dec decimal(4,2),  
            bin binary)
        row format delimited
        fields terminated by '|';
 
-load data local inpath '../data/files/over10k' into table over10k;
+load data local inpath '../../data/files/over10k' into table over10k;
 
 select p_mfgr, p_retailprice, p_size,
 round(sum(p_retailprice) over w1 , 2) = round(sum(lag(p_retailprice,1,0.0)) over w1 + last_value(p_retailprice) over w1 , 2), 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q
index bb371e900975b868a80b94d9ac78485c06a69d32..1c6e1aac37a1a5861f79eb59b802cd22cb288044 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q
@@ -10,12 +10,12 @@ create table over10k(
            bo boolean,
            s string,
 	   ts timestamp, 
-           dec decimal,  
+           dec decimal(4,2),  
            bin binary)
        row format delimited
        fields terminated by '|';
 
-load data local inpath '../data/files/over10k' into table over10k;
+load data local inpath '../../data/files/over10k' into table over10k;
 
 select s, rank() over (partition by s order by si), sum(b) over (partition by s order by si) from over10k limit 100;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q
index 8a9d0012593e6d70112cbed5f1ec80b2de130727..05da2ba7efeafabd72809c181c3d4466fb20a6c9 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q
@@ -9,13 +9,13 @@ create table over10k(
            d double,
            bo boolean,
            s string,
-	   ts timestamp, 
-           dec decimal,  
+           ts timestamp, 
+           dec decimal(4,2),  
            bin binary)
        row format delimited
        fields terminated by '|';
 
-load data local inpath '../data/files/over10k' into table over10k;
+load data local inpath '../../data/files/over10k' into table over10k;
 
 select s, row_number() over (partition by d order by dec) from over10k limit 100;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q
index 505c259f4b7b3fa3c7300626f7f74515bd9b1b0c..73e8192ee6e4c0d20b7b3aba78734fc394232fbd 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q
@@ -10,12 +10,12 @@ create table over10k(
            bo boolean,
            s string,
 	   ts timestamp, 
-           dec decimal,  
+           dec decimal(4,2),  
            bin binary)
        row format delimited
        fields terminated by '|';
 
-load data local inpath '../data/files/over10k' into table over10k;
+load data local inpath '../../data/files/over10k' into table over10k;
 
 select i, ntile(10) over (partition by s order by i) from over10k limit 100;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q
index bf76867813e4e22ae6c120c8bf0f3297c83a5194..4b951179e09e6c86ab34e9506a13b3b24578701c 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q
@@ -10,12 +10,12 @@ create table over10k(
            bo boolean,
            s string,
 	   ts timestamp, 
-           dec decimal,  
+           dec decimal(4,2),  
            bin binary)
        row format delimited
        fields terminated by '|';
 
-load data local inpath '../data/files/over10k' into table over10k;
+load data local inpath '../../data/files/over10k' into table over10k;
 
 select s, rank() over (partition by f order by t) from over10k limit 100;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q
index f22b992cd4386d75e3829745a8926f9d2ac56d83..0173ab7a3ac5657d05640ace66d6a11fd458ae75 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q
@@ -15,7 +15,7 @@ create table over10k(
        row format delimited
        fields terminated by '|';
 
-load data local inpath '../data/files/over10k' into table over10k;
+load data local inpath '../../data/files/over10k' into table over10k;
 
 select s, min(i) over (partition by s) from over10k limit 100;
 
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf2.q
new file mode 100644
index 0000000000000000000000000000000000000000..b813657baee4716ecba94ed4c90f8a2efe7bf11b
--- /dev/null
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf2.q
@@ -0,0 +1,4 @@
+-- user-added aggregates should be usable as windowing functions
+create temporary function mysum as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum';
+
+select sum(key) over (), mysum(key) over () from src limit 1;
diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q
index 7cc1367306956c5e4412f00015975a27bbae3276..6d8ce670454d086aabd483c4c16e12df80cdea10 100644
--- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q
+++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q
@@ -15,7 +15,7 @@ create table over10k(
        row format delimited
        fields terminated by '|';
 
-load data local inpath '../data/files/over10k' into table over10k;
+load data local inpath '../../data/files/over10k' into table over10k;
 
 select s, sum(b) over (partition by i order by s,b rows unbounded preceding) from over10k limit 100;
 
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
index aaefe84ce81eacbbc0a948b46afc2637265a6592..a90fc023e67d8843dc517e3acf0aff0444060c77 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala
@@ -21,11 +21,9 @@ import org.scalatest.BeforeAndAfterAll
 
 import scala.reflect.ClassTag
 
-
 import org.apache.spark.sql.{SQLConf, QueryTest}
 import org.apache.spark.sql.catalyst.plans.logical.NativeCommand
 import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin}
-import org.apache.spark.sql.hive.HiveShim
 import org.apache.spark.sql.hive.test.TestHive
 import org.apache.spark.sql.hive.test.TestHive._
 
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala
index 46b11b582b26d94657b1d8d83717158d229ac240..ca78dfba4fa38d45414dba8ac1008dae5d51e53b 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala
@@ -22,7 +22,7 @@ import scala.util.Try
 import org.scalatest.FunSuite
 
 import org.apache.spark.api.java.JavaSparkContext
-import org.apache.spark.sql.api.java.JavaSchemaRDD
+import org.apache.spark.sql.api.java.{JavaSQLContext, JavaSchemaRDD}
 import org.apache.spark.sql.execution.ExplainCommand
 import org.apache.spark.sql.hive.test.TestHive
 
@@ -33,9 +33,7 @@ class JavaHiveQLSuite extends FunSuite {
   lazy val javaCtx = new JavaSparkContext(TestHive.sparkContext)
 
   // There is a little trickery here to avoid instantiating two HiveContexts in the same JVM
-  lazy val javaHiveCtx = new JavaHiveContext(javaCtx) {
-    override val sqlContext = TestHive
-  }
+  lazy val javaHiveCtx = new JavaHiveContext(TestHive)
 
   test("SELECT * FROM src") {
     assert(
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
index 79cc7a3fcc7d694666683201c444ffd86e85d60d..44eb4cfa593357187c3895d8238e6430a5167e42 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
@@ -142,14 +142,25 @@ abstract class HiveComparisonTest
       // Hack: Hive simply prints the result of a SET command to screen,
       // and does not return it as a query answer.
       case _: SetCommand => Seq("0")
+      case LogicalNativeCommand(c) if c.toLowerCase.contains("desc") =>
+        answer
+          .filterNot(nonDeterministicLine)
+          .map(_.replaceAll("from deserializer", ""))
+          .map(_.replaceAll("None", ""))
+          .map(_.trim)
+          .filterNot(_ == "")
       case _: LogicalNativeCommand => answer.filterNot(nonDeterministicLine).filterNot(_ == "")
       case _: ExplainCommand => answer
       case _: DescribeCommand =>
         // Filter out non-deterministic lines and lines which do not have actual results but
         // can introduce problems because of the way Hive formats these lines.
         // Then, remove empty lines. Do not sort the results.
-        answer.filterNot(
-          r => nonDeterministicLine(r) || ignoredLine(r)).map(_.trim).filterNot(_ == "")
+        answer
+          .filterNot(r => nonDeterministicLine(r) || ignoredLine(r))
+          .map(_.replaceAll("from deserializer", ""))
+          .map(_.replaceAll("None", ""))
+          .map(_.trim)
+          .filterNot(_ == "")
       case plan => if (isSorted(plan)) answer else answer.sorted
     }
     orderedAnswer.map(cleanPaths)
@@ -164,6 +175,7 @@ abstract class HiveComparisonTest
     "last_modified_by",
     "last_modified_time",
     "Owner:",
+    "COLUMN_STATS_ACCURATE",
     // The following are hive specific schema parameters which we do not need to match exactly.
     "numFiles",
     "numRows",
@@ -237,6 +249,7 @@ abstract class HiveComparisonTest
           // the system to return the wrong answer.  Since we have no intention of mirroring their
           // previously broken behavior we simply filter out changes to this setting.
           .filterNot(_ contains "hive.outerjoin.supports.filters")
+          .filterNot(_ contains "hive.exec.post.hooks")
 
       if (allQueries != queryList)
         logWarning(s"Simplifications made on unsupported operations for test $testCaseName")
@@ -345,7 +358,7 @@ abstract class HiveComparisonTest
         (queryList, hiveResults, catalystResults).zipped.foreach {
           case (query, hive, (hiveQuery, catalyst)) =>
             // Check that the results match unless its an EXPLAIN query.
-            val preparedHive = prepareAnswer(hiveQuery,hive)
+            val preparedHive = prepareAnswer(hiveQuery, hive)
 
             if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && preparedHive != catalyst) {
 
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 5de20175d9f576c959fec60c412cca27e8cbe223..322a25bb20837ddae7ab616f976fc019dd521aa0 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
@@ -508,19 +508,19 @@ class HiveQuerySuite extends HiveComparisonTest {
     // Describe a partition is a native command
     assertResult(
       Array(
-        Array("key", "int", HiveShim.getEmptyCommentsFieldValue),
-        Array("value", "string", HiveShim.getEmptyCommentsFieldValue),
-        Array("dt", "string", HiveShim.getEmptyCommentsFieldValue),
-        Array("", "", ""),
-        Array("# Partition Information", "", ""),
+        Array("key", "int"),
+        Array("value", "string"),
+        Array("dt", "string"),
+        Array(""),
+        Array("# Partition Information"),
         Array("# col_name", "data_type", "comment"),
-        Array("", "", ""),
-        Array("dt", "string", HiveShim.getEmptyCommentsFieldValue))
+        Array(""),
+        Array("dt", "string"))
     ) {
       sql("DESCRIBE test_describe_commands1 PARTITION (dt='2008-06-08')")
         .select('result)
         .collect()
-        .map(_.getString(0).split("\t").map(_.trim))
+        .map(_.getString(0).replaceAll("None", "").trim.split("\t").map(_.trim))
     }
 
     // Describe a registered temporary table.
diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala
deleted file mode 100644
index 6dde636965afdde84a8aa6c7616047b82c424776..0000000000000000000000000000000000000000
--- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.hive
-
-import java.net.URI
-import java.util.{ArrayList => JArrayList}
-import java.util.Properties
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.hive.common.`type`.HiveDecimal
-import org.apache.hadoop.hive.conf.HiveConf
-import org.apache.hadoop.hive.ql.Context
-import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table}
-import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc}
-import org.apache.hadoop.hive.ql.processors._
-import org.apache.hadoop.hive.ql.stats.StatsSetupConst
-import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils}
-import org.apache.hadoop.{io => hadoopIo}
-import org.apache.hadoop.mapred.InputFormat
-import scala.collection.JavaConversions._
-import scala.language.implicitConversions
-
-/**
- * A compatibility layer for interacting with Hive version 0.12.0.
- */
-private[hive] object HiveShim {
-  val version = "0.12.0"
-  val metastoreDecimal = "decimal"
-
-  def getTableDesc(
-    serdeClass: Class[_ <: Deserializer],
-    inputFormatClass: Class[_ <: InputFormat[_, _]],
-    outputFormatClass: Class[_],
-    properties: Properties) = {
-    new TableDesc(serdeClass, inputFormatClass, outputFormatClass, properties)
-  }
-
-  def createDriverResultsArray = new JArrayList[String]
-
-  def processResults(results: JArrayList[String]) = results
-
-  def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE
-
-  def createDefaultDBIfNeeded(context: HiveContext) = {  }
-
-  /** The string used to denote an empty comments field in the schema. */
-  def getEmptyCommentsFieldValue = "None"
-
-  def getCommandProcessor(cmd: Array[String], conf: HiveConf) = {
-    CommandProcessorFactory.get(cmd(0), conf)
-  }
-
-  def createDecimal(bd: java.math.BigDecimal): HiveDecimal = {
-    new HiveDecimal(bd)
-  }
-
-  def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) {
-    ColumnProjectionUtils.appendReadColumnIDs(conf, ids)
-    ColumnProjectionUtils.appendReadColumnNames(conf, names)
-  }
-
-  def getExternalTmpPath(context: Context, uri: URI) = {
-    context.getExternalTmpFileURI(uri)
-  }
-
-  def getDataLocationPath(p: Partition) = p.getPartitionPath
-
-  def getAllPartitionsOf(client: Hive, tbl: Table) =  client.getAllPartitionsForPruner(tbl)
-
-}
-
-class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean)
-  extends FileSinkDesc(dir, tableInfo, compressed) {
-}
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
new file mode 100644
index 0000000000000000000000000000000000000000..2317d2e76341f304c9def956b609613b0c5ac9a1
--- /dev/null
+++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive
+
+import java.net.URI
+import java.util.{ArrayList => JArrayList}
+import java.util.Properties
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.hive.common.`type`.HiveDecimal
+import org.apache.hadoop.hive.conf.HiveConf
+import org.apache.hadoop.hive.ql.Context
+import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table}
+import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc}
+import org.apache.hadoop.hive.ql.processors._
+import org.apache.hadoop.hive.ql.stats.StatsSetupConst
+import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils}
+import org.apache.hadoop.{io => hadoopIo}
+import org.apache.hadoop.mapred.InputFormat
+import scala.collection.JavaConversions._
+import scala.language.implicitConversions
+
+/**
+ * A compatibility layer for interacting with Hive version 0.12.0.
+ */
+private[hive] object HiveShim {
+  val version = "0.12.0"
+  val metastoreDecimal = "decimal"
+
+  def getTableDesc(
+    serdeClass: Class[_ <: Deserializer],
+    inputFormatClass: Class[_ <: InputFormat[_, _]],
+    outputFormatClass: Class[_],
+    properties: Properties) = {
+    new TableDesc(serdeClass, inputFormatClass, outputFormatClass, properties)
+  }
+
+  def createDriverResultsArray = new JArrayList[String]
+
+  def processResults(results: JArrayList[String]) = results
+
+  def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE
+
+  def createDefaultDBIfNeeded(context: HiveContext) = {  }
+
+  def getCommandProcessor(cmd: Array[String], conf: HiveConf) = {
+    CommandProcessorFactory.get(cmd(0), conf)
+  }
+
+  def createDecimal(bd: java.math.BigDecimal): HiveDecimal = {
+    new HiveDecimal(bd)
+  }
+
+  def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) {
+    ColumnProjectionUtils.appendReadColumnIDs(conf, ids)
+    ColumnProjectionUtils.appendReadColumnNames(conf, names)
+  }
+
+  def getExternalTmpPath(context: Context, uri: URI) = {
+    context.getExternalTmpFileURI(uri)
+  }
+
+  def getDataLocationPath(p: Partition) = p.getPartitionPath
+
+  def getAllPartitionsOf(client: Hive, tbl: Table) =  client.getAllPartitionsForPruner(tbl)
+
+  def compatibilityBlackList = Seq(
+    "decimal_.*",
+    "drop_partitions_filter2",
+    "show_.*",
+    "serde_regex",
+    "udf_to_date",
+    "udaf_collect_set",
+    "udf_concat"
+  )
+
+}
+
+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/Shim.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala
deleted file mode 100644
index 8678c0c475db4001f830aed6ab6e3d50da02c3a6..0000000000000000000000000000000000000000
--- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala
+++ /dev/null
@@ -1,170 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.hive
-
-import java.util.{ArrayList => JArrayList}
-import java.util.Properties
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.hive.common.StatsSetupConst
-import org.apache.hadoop.hive.common.`type`.{HiveDecimal}
-import org.apache.hadoop.hive.conf.HiveConf
-import org.apache.hadoop.hive.ql.Context
-import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition}
-import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc}
-import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory
-import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer}
-import org.apache.hadoop.mapred.InputFormat
-import org.apache.spark.Logging
-import org.apache.hadoop.{io => hadoopIo}
-import scala.collection.JavaConversions._
-import scala.language.implicitConversions
-
-/**
- * A compatibility layer for interacting with Hive version 0.13.1.
- */
-private[hive] object HiveShim {
-  val version = "0.13.1"
-  /*
-   * TODO: hive-0.13 support DECIMAL(precision, scale), DECIMAL in hive-0.12 is actually DECIMAL(38,unbounded)
-   * Full support of new decimal feature need to be fixed in seperate PR.
-   */
-  val metastoreDecimal = "decimal\\((\\d+),(\\d+)\\)".r
-
-  def getTableDesc(
-    serdeClass: Class[_ <: Deserializer],
-    inputFormatClass: Class[_ <: InputFormat[_, _]],
-    outputFormatClass: Class[_],
-    properties: Properties) = {
-    new TableDesc(inputFormatClass, outputFormatClass, properties)
-  }
-
-  def createDriverResultsArray = new JArrayList[Object]
-
-  def processResults(results: JArrayList[Object]) = {
-    results.map { r =>
-      r match {
-        case s: String => s
-        case a: Array[Object] => a(0).asInstanceOf[String]
-      }
-    }
-  }
-
-  def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE
-
-  def createDefaultDBIfNeeded(context: HiveContext) = {
-    context.runSqlHive("CREATE DATABASE default")
-    context.runSqlHive("USE default")
-  }
-
-  /* The string used to denote an empty comments field in the schema. */
-  def getEmptyCommentsFieldValue = ""
-
-  def getCommandProcessor(cmd: Array[String], conf: HiveConf) = {
-    CommandProcessorFactory.get(cmd, conf)
-  }
-
-  def createDecimal(bd: java.math.BigDecimal): HiveDecimal = {
-    HiveDecimal.create(bd)
-  }
-
-  /*
-   * This function in hive-0.13 become private, but we have to do this to walkaround hive bug
-   */
-  private def appendReadColumnNames(conf: Configuration, cols: Seq[String]) {
-    val old: String = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "")
-    val result: StringBuilder = new StringBuilder(old)
-    var first: Boolean = old.isEmpty
-
-    for (col <- cols) {
-      if (first) {
-        first = false
-      } else {
-        result.append(',')
-      }
-      result.append(col)
-    }
-    conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, result.toString)
-  }
-
-  /*
-   * Cannot use ColumnProjectionUtils.appendReadColumns directly, if ids is null or empty
-   */
-  def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) {
-    if (ids != null && ids.size > 0) {
-      ColumnProjectionUtils.appendReadColumns(conf, ids)
-    }
-    if (names != null && names.size > 0) {
-      appendReadColumnNames(conf, names)
-    }
-  }
-
-  def getExternalTmpPath(context: Context, path: Path) = {
-    context.getExternalTmpPath(path.toUri)
-  }
-
-  def getDataLocationPath(p: Partition) = p.getDataLocation
-
-  def getAllPartitionsOf(client: Hive, tbl: Table) =  client.getAllPartitionsOf(tbl)
-
-  /*
-   * Bug introdiced in hive-0.13. FileSinkDesc is serializable, but its member path is not.
-   * Fix it through wrapper.
-   * */
-  implicit def wrapperToFileSinkDesc(w: ShimFileSinkDesc): FileSinkDesc = {
-    var f = new FileSinkDesc(new Path(w.dir), w.tableInfo, w.compressed)
-    f.setCompressCodec(w.compressCodec)
-    f.setCompressType(w.compressType)
-    f.setTableInfo(w.tableInfo)
-    f.setDestTableId(w.destTableId)
-    f
-  }
-}
-
-/*
- * Bug introdiced 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)
-  extends Serializable with Logging {
-  var compressCodec: String = _
-  var compressType: String = _
-  var destTableId: Int = _
-
-  def setCompressed(compressed: Boolean) {
-    this.compressed = compressed
-  }
-
-  def getDirName = dir
-
-  def setDestTableId(destTableId: Int) {
-    this.destTableId = destTableId
-  }
-
-  def setTableInfo(tableInfo: TableDesc) {
-    this.tableInfo = tableInfo
-  }
-
-  def setCompressCodec(intermediateCompressorCodec: String) {
-    compressCodec = intermediateCompressorCodec
-  }
-
-  def setCompressType(intermediateCompressType: String) {
-    compressType = intermediateCompressType
-  }
-}
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
new file mode 100644
index 0000000000000000000000000000000000000000..b8d893d8c131989934fd32c89e902626aa31d4c1
--- /dev/null
+++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive
+
+import java.util.{ArrayList => JArrayList}
+import java.util.Properties
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.hive.common.StatsSetupConst
+import org.apache.hadoop.hive.common.`type`.{HiveDecimal}
+import org.apache.hadoop.hive.conf.HiveConf
+import org.apache.hadoop.hive.ql.Context
+import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition}
+import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc}
+import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory
+import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer}
+import org.apache.hadoop.mapred.InputFormat
+import org.apache.spark.Logging
+import org.apache.hadoop.{io => hadoopIo}
+import scala.collection.JavaConversions._
+import scala.language.implicitConversions
+
+/**
+ * A compatibility layer for interacting with Hive version 0.13.1.
+ */
+private[hive] object HiveShim {
+  val version = "0.13.1"
+  /*
+   * TODO: hive-0.13 support DECIMAL(precision, scale), DECIMAL in hive-0.12 is actually DECIMAL(38,unbounded)
+   * Full support of new decimal feature need to be fixed in seperate PR.
+   */
+  val metastoreDecimal = "decimal\\((\\d+),(\\d+)\\)".r
+
+  def getTableDesc(
+    serdeClass: Class[_ <: Deserializer],
+    inputFormatClass: Class[_ <: InputFormat[_, _]],
+    outputFormatClass: Class[_],
+    properties: Properties) = {
+    new TableDesc(inputFormatClass, outputFormatClass, properties)
+  }
+
+  def createDriverResultsArray = new JArrayList[Object]
+
+  def processResults(results: JArrayList[Object]) = {
+    results.map { r =>
+      r match {
+        case s: String => s
+        case a: Array[Object] => a(0).asInstanceOf[String]
+      }
+    }
+  }
+
+  def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE
+
+  def createDefaultDBIfNeeded(context: HiveContext) = {
+    context.runSqlHive("CREATE DATABASE default")
+    context.runSqlHive("USE default")
+  }
+
+  def getCommandProcessor(cmd: Array[String], conf: HiveConf) = {
+    CommandProcessorFactory.get(cmd, conf)
+  }
+
+  def createDecimal(bd: java.math.BigDecimal): HiveDecimal = {
+    HiveDecimal.create(bd)
+  }
+
+  /*
+   * This function in hive-0.13 become private, but we have to do this to walkaround hive bug
+   */
+  private def appendReadColumnNames(conf: Configuration, cols: Seq[String]) {
+    val old: String = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "")
+    val result: StringBuilder = new StringBuilder(old)
+    var first: Boolean = old.isEmpty
+
+    for (col <- cols) {
+      if (first) {
+        first = false
+      } else {
+        result.append(',')
+      }
+      result.append(col)
+    }
+    conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, result.toString)
+  }
+
+  /*
+   * Cannot use ColumnProjectionUtils.appendReadColumns directly, if ids is null or empty
+   */
+  def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) {
+    if (ids != null && ids.size > 0) {
+      ColumnProjectionUtils.appendReadColumns(conf, ids)
+    }
+    if (names != null && names.size > 0) {
+      appendReadColumnNames(conf, names)
+    }
+  }
+
+  def getExternalTmpPath(context: Context, path: Path) = {
+    context.getExternalTmpPath(path.toUri)
+  }
+
+  def getDataLocationPath(p: Partition) = p.getDataLocation
+
+  def getAllPartitionsOf(client: Hive, tbl: Table) =  client.getAllPartitionsOf(tbl)
+
+  def compatibilityBlackList = Seq()
+
+  /*
+   * Bug introdiced in hive-0.13. FileSinkDesc is serializable, but its member path is not.
+   * Fix it through wrapper.
+   * */
+  implicit def wrapperToFileSinkDesc(w: ShimFileSinkDesc): FileSinkDesc = {
+    var f = new FileSinkDesc(new Path(w.dir), w.tableInfo, w.compressed)
+    f.setCompressCodec(w.compressCodec)
+    f.setCompressType(w.compressType)
+    f.setTableInfo(w.tableInfo)
+    f.setDestTableId(w.destTableId)
+    f
+  }
+}
+
+/*
+ * Bug introdiced 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)
+  extends Serializable with Logging {
+  var compressCodec: String = _
+  var compressType: String = _
+  var destTableId: Int = _
+
+  def setCompressed(compressed: Boolean) {
+    this.compressed = compressed
+  }
+
+  def getDirName = dir
+
+  def setDestTableId(destTableId: Int) {
+    this.destTableId = destTableId
+  }
+
+  def setTableInfo(tableInfo: TableDesc) {
+    this.tableInfo = tableInfo
+  }
+
+  def setCompressCodec(intermediateCompressorCodec: String) {
+    compressCodec = intermediateCompressorCodec
+  }
+
+  def setCompressType(intermediateCompressType: String) {
+    compressType = intermediateCompressType
+  }
+}